You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2020/08/31 23:53:44 UTC

[GitHub] [arrow] kszucs opened a new pull request #8088: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API [WIP]

kszucs opened a new pull request #8088:
URL: https://github.com/apache/arrow/pull/8088


   


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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,

Review comment:
       Updating.

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,
+                            std::shared_ptr<ArrayBuilder> builder,
+                            const std::vector<std::shared_ptr<Self>>& children,
+                            OptionsType options) {
+    type_ = std::move(type);
+    builder_ = std::move(builder);
+    children_ = std::move(children);
+    options_ = std::move(options);
+    return Init();
+  }
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>>& children() const { return children_; }
+
+  Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }

Review comment:
       Giving access to the mapping trait would require the implementor handle another template parameter. I'm updating it, but not sure that's desirable.

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,
+                            std::shared_ptr<ArrayBuilder> builder,
+                            const std::vector<std::shared_ptr<Self>>& children,
+                            OptionsType options) {
+    type_ = std::move(type);
+    builder_ = std::move(builder);
+    children_ = std::move(children);
+    options_ = std::move(options);
+    return Init();
+  }
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>>& children() const { return children_; }
+
+  Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }

Review comment:
       Giving access to the mapping trait would require the implementor handle another template parameter. 
   I'd also need to have specific implementations for Null and Map types.

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,
+                            std::shared_ptr<ArrayBuilder> builder,
+                            const std::vector<std::shared_ptr<Self>>& children,
+                            OptionsType options) {
+    type_ = std::move(type);
+    builder_ = std::move(builder);
+    children_ = std::move(children);
+    options_ = std::move(options);
+    return Init();
+  }
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>>& children() const { return children_; }
+
+  Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }

Review comment:
       Updated.

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,
+                            std::shared_ptr<ArrayBuilder> builder,
+                            const std::vector<std::shared_ptr<Self>>& children,
+                            OptionsType options) {
+    type_ = std::move(type);
+    builder_ = std::move(builder);
+    children_ = std::move(children);
+    options_ = std::move(options);
+    return Init();
+  }
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>>& children() const { return children_; }
+
+  Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {

Review comment:
       Updated.

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,
+                            std::shared_ptr<ArrayBuilder> builder,
+                            const std::vector<std::shared_ptr<Self>>& children,
+                            OptionsType options) {
+    type_ = std::move(type);
+    builder_ = std::move(builder);
+    children_ = std::move(children);
+    options_ = std::move(options);
+    return Init();
+  }
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>>& children() const { return children_; }
+
+  Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter, template <typename...> class ConverterTrait>
+struct MakeConverterImpl;
+
+template <typename Converter, template <typename...> class ConverterTrait>
+static Result<std::shared_ptr<Converter>> MakeConverter(
+    std::shared_ptr<DataType> type, MemoryPool* pool,
+    typename Converter::OptionsType options) {
+  std::shared_ptr<Converter> out;
+  MakeConverterImpl<Converter, ConverterTrait> visitor = {type, pool, options, &out};
+  ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+  return out;
+}

Review comment:
       Updated.

##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -329,985 +302,596 @@ struct ValueConverter<DurationType> {
         default:
           return Status::UnknownError("Invalid time unit");
       }
+    } else if (PyArray_CheckAnyScalarExact(obj)) {
+      // validate that the numpy scalar has np.datetime64 dtype
+      std::shared_ptr<DataType> numpy_type;
+      RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &numpy_type));
+      if (!numpy_type->Equals(*type)) {
+        return Status::NotImplemented("Expected np.timedelta64 but got: ",
+                                      numpy_type->ToString());
+      }
+      return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
     } else {
       RETURN_NOT_OK(internal::CIntFromPython(obj, &value));
     }
     return value;
   }
 
-  static inline Result<int64_t> FromNumpy(PyObject* obj, TimeUnit::type unit) {
-    // validate that the numpy scalar has np.timedelta64 dtype
-    std::shared_ptr<DataType> type;
-    RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &type));
-    if (type->id() != DurationType::type_id) {
-      // TODO(kszucs): the message should highlight the received numpy dtype
-      return Status::Invalid("Expected np.timedelta64 but got: ", type->ToString());
-    }
-    // validate that the time units are matching
-    if (unit != checked_cast<const DurationType&>(*type).unit()) {
-      return Status::NotImplemented(
-          "Cannot convert NumPy np.timedelta64 objects with differing unit");
-    }
-    // convert the numpy value
-    return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_any_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    return std::move(view);
-  }
-};
+  // The binary-like intermediate representation is PyBytesView because it keeps temporary
+  // python objects alive (non-contiguous memoryview) and stores whether the original
+  // object was unicode encoded or not, which is used for unicode -> bytes coersion if
+  // there is a non-unicode object observed.
 
-template <typename Type>
-struct ValueConverter<Type, enable_if_string_like<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    // strict conversion, force output to be unicode / utf8 and validate that
-    // any binary values are utf8
-    bool is_utf8 = false;
-    PyBytesView view;
-
-    RETURN_NOT_OK(view.FromString(obj, &is_utf8));
-    if (!is_utf8) {
-      return internal::InvalidValue(obj, "was not a utf8 string");
-    }
-    return std::move(view);
+  static Result<PyBytesView> Convert(const BaseBinaryType*, const O&, I obj) {
+    return PyBytesView::FromString(obj);
   }
 
-  static inline Result<PyBytesView> FromPython(PyObject* obj, bool* is_utf8) {
-    PyBytesView view;
-
-    // Non-strict conversion; keep track of whether values are unicode or bytes
-    if (PyUnicode_Check(obj)) {
-      *is_utf8 = true;
-      RETURN_NOT_OK(view.FromUnicode(obj));
+  static Result<PyBytesView> Convert(const FixedSizeBinaryType* type, const O&, I obj) {
+    ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj));
+    if (ARROW_PREDICT_TRUE(view.size == type->byte_width())) {
+      return std::move(view);
     } else {
-      // If not unicode or bytes, FromBinary will error
-      *is_utf8 = false;
-      RETURN_NOT_OK(view.FromBinary(obj));
-    }
-    return std::move(view);
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_fixed_size_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj, int32_t byte_width) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    if (ARROW_PREDICT_FALSE(view.size != byte_width)) {
       std::stringstream ss;
-      ss << "expected to be length " << byte_width << " was " << view.size;
+      ss << "expected to be length " << type->byte_width() << " was " << view.size;
       return internal::InvalidValue(obj, ss.str());
-    } else {
-      return std::move(view);
     }
   }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter base and CRTP "middle" subclasses
-
-class SeqConverter;
-
-// Forward-declare converter factory
-Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
-                    bool strict_conversions, bool ignore_timezone,
-                    std::unique_ptr<SeqConverter>* out);
-
-// Marshal Python sequence (list, tuple, etc.) to Arrow array
-class SeqConverter {
- public:
-  virtual ~SeqConverter() = default;
-
-  // Initialize the sequence converter with an ArrayBuilder created
-  // externally. The reason for this interface is that we have
-  // arrow::MakeBuilder which also creates child builders for nested types, so
-  // we have to pass in the child builders to child SeqConverter in the case of
-  // converting Python objects to Arrow nested types
-  virtual Status Init(ArrayBuilder* builder) = 0;
 
-  // Append a single null value to the builder
-  virtual Status AppendNull() = 0;
-
-  // Append a valid value
-  virtual Status AppendValue(PyObject* seq) = 0;
-
-  // Append a single python object handling Null values
-  virtual Status Append(PyObject* seq) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status Extend(PyObject* seq, int64_t size) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status ExtendMasked(PyObject* seq, PyObject* mask, int64_t size) = 0;
-
-  virtual Status Close() {
-    if (chunks_.size() == 0 || builder_->length() > 0) {
-      std::shared_ptr<Array> last_chunk;
-      RETURN_NOT_OK(builder_->Finish(&last_chunk));
-      chunks_.emplace_back(std::move(last_chunk));
+  template <typename T>
+  static enable_if_string<T, Result<PyBytesView>> Convert(const T*, const O& options,
+                                                          I obj) {
+    if (options.strict) {
+      // Strict conversion, force output to be unicode / utf8 and validate that
+      // any binary values are utf8
+      ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj, true));
+      if (!view.is_utf8) {
+        return internal::InvalidValue(obj, "was not a utf8 string");
+      }
+      return std::move(view);
+    } else {
+      // Non-strict conversion; keep track of whether values are unicode or bytes
+      return PyBytesView::FromString(obj);
     }
-    return Status::OK();
   }
 
-  virtual Status GetResult(std::shared_ptr<ChunkedArray>* out) {
-    // Still some accumulated data in the builder. If there are no chunks, we
-    // always call Finish to deal with the edge case where a size-0 sequence
-    // was converted with a specific output type, like array([], type=t)
-    RETURN_NOT_OK(Close());
-    *out = std::make_shared<ChunkedArray>(this->chunks_, builder_->type());
-    return Status::OK();
+  static Result<bool> Convert(const DataType* type, const O&, I obj) {
+    return Status::NotImplemented("PyValue::Convert is not implemented for type ", type);
   }
-
-  ArrayBuilder* builder() const { return builder_; }
-
-  int num_chunks() const { return static_cast<int>(chunks_.size()); }
-
- protected:
-  ArrayBuilder* builder_;
-  bool unfinished_builder_;
-  std::vector<std::shared_ptr<Array>> chunks_;
 };
 
-template <typename Type, NullCoding null_coding = NullCoding::NONE_ONLY>
-class TypedConverter : public SeqConverter {
- public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  Status Init(ArrayBuilder* builder) override {
-    builder_ = builder;
-    DCHECK_NE(builder_, nullptr);
-    typed_builder_ = checked_cast<BuilderType*>(builder);
-    return Status::OK();
-  }
-
-  // Append a missing item (default implementation)
-  Status AppendNull() override { return this->typed_builder_->AppendNull(); }
-
-  // Append null if the obj is None or pandas null otherwise the valid value
-  Status Append(PyObject* obj) override {
-    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
-  }
-
-  Status Extend(PyObject* obj, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequence(
-        obj, [this](PyObject* item, bool* /* unused */) { return this->Append(item); });
-  }
+template <typename T>
+Status Extend(T* converter, PyObject* values, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequence(values, [converter](PyObject* item, bool* /* unused */) {
+    return converter->Append(item);
+  });
+}
 
-  Status ExtendMasked(PyObject* obj, PyObject* mask, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequenceMasked(
-        obj, mask, [this](PyObject* item, bool is_masked, bool* /* unused */) {
-          if (is_masked) {
-            return this->AppendNull();
-          } else {
-            // This will also apply the null-checking convention in the event
-            // that the value is not masked
-            return this->Append(item);  // perhaps use AppendValue instead?
-          }
-        });
-  }
+// Convert and append a sequence of values masked with a numpy array
+template <typename T>
+Status ExtendMasked(T* converter, PyObject* values, PyObject* mask, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequenceMasked(
+      values, mask, [converter](PyObject* item, bool is_masked, bool* /* unused */) {
+        if (is_masked) {
+          return converter->AppendNull();
+        } else {
+          // This will also apply the null-checking convention in the event
+          // that the value is not masked
+          return converter->Append(item);  // perhaps use AppendValue instead?
+        }
+      });
+}
 
- protected:
-  BuilderType* typed_builder_;
-};
+// The base Converter class is a mixin with predefined behavior and constructors.
+class PyConverter : public Converter<PyObject*, PyConversionOptions, PyConverter> {};

Review comment:
       Updated.

##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -329,985 +302,596 @@ struct ValueConverter<DurationType> {
         default:
           return Status::UnknownError("Invalid time unit");
       }
+    } else if (PyArray_CheckAnyScalarExact(obj)) {
+      // validate that the numpy scalar has np.datetime64 dtype
+      std::shared_ptr<DataType> numpy_type;
+      RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &numpy_type));
+      if (!numpy_type->Equals(*type)) {
+        return Status::NotImplemented("Expected np.timedelta64 but got: ",
+                                      numpy_type->ToString());
+      }
+      return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
     } else {
       RETURN_NOT_OK(internal::CIntFromPython(obj, &value));
     }
     return value;
   }
 
-  static inline Result<int64_t> FromNumpy(PyObject* obj, TimeUnit::type unit) {
-    // validate that the numpy scalar has np.timedelta64 dtype
-    std::shared_ptr<DataType> type;
-    RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &type));
-    if (type->id() != DurationType::type_id) {
-      // TODO(kszucs): the message should highlight the received numpy dtype
-      return Status::Invalid("Expected np.timedelta64 but got: ", type->ToString());
-    }
-    // validate that the time units are matching
-    if (unit != checked_cast<const DurationType&>(*type).unit()) {
-      return Status::NotImplemented(
-          "Cannot convert NumPy np.timedelta64 objects with differing unit");
-    }
-    // convert the numpy value
-    return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_any_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    return std::move(view);
-  }
-};
+  // The binary-like intermediate representation is PyBytesView because it keeps temporary
+  // python objects alive (non-contiguous memoryview) and stores whether the original
+  // object was unicode encoded or not, which is used for unicode -> bytes coersion if
+  // there is a non-unicode object observed.
 
-template <typename Type>
-struct ValueConverter<Type, enable_if_string_like<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    // strict conversion, force output to be unicode / utf8 and validate that
-    // any binary values are utf8
-    bool is_utf8 = false;
-    PyBytesView view;
-
-    RETURN_NOT_OK(view.FromString(obj, &is_utf8));
-    if (!is_utf8) {
-      return internal::InvalidValue(obj, "was not a utf8 string");
-    }
-    return std::move(view);
+  static Result<PyBytesView> Convert(const BaseBinaryType*, const O&, I obj) {
+    return PyBytesView::FromString(obj);
   }
 
-  static inline Result<PyBytesView> FromPython(PyObject* obj, bool* is_utf8) {
-    PyBytesView view;
-
-    // Non-strict conversion; keep track of whether values are unicode or bytes
-    if (PyUnicode_Check(obj)) {
-      *is_utf8 = true;
-      RETURN_NOT_OK(view.FromUnicode(obj));
+  static Result<PyBytesView> Convert(const FixedSizeBinaryType* type, const O&, I obj) {
+    ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj));
+    if (ARROW_PREDICT_TRUE(view.size == type->byte_width())) {
+      return std::move(view);
     } else {
-      // If not unicode or bytes, FromBinary will error
-      *is_utf8 = false;
-      RETURN_NOT_OK(view.FromBinary(obj));
-    }
-    return std::move(view);
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_fixed_size_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj, int32_t byte_width) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    if (ARROW_PREDICT_FALSE(view.size != byte_width)) {
       std::stringstream ss;
-      ss << "expected to be length " << byte_width << " was " << view.size;
+      ss << "expected to be length " << type->byte_width() << " was " << view.size;
       return internal::InvalidValue(obj, ss.str());
-    } else {
-      return std::move(view);
     }
   }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter base and CRTP "middle" subclasses
-
-class SeqConverter;
-
-// Forward-declare converter factory
-Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
-                    bool strict_conversions, bool ignore_timezone,
-                    std::unique_ptr<SeqConverter>* out);
-
-// Marshal Python sequence (list, tuple, etc.) to Arrow array
-class SeqConverter {
- public:
-  virtual ~SeqConverter() = default;
-
-  // Initialize the sequence converter with an ArrayBuilder created
-  // externally. The reason for this interface is that we have
-  // arrow::MakeBuilder which also creates child builders for nested types, so
-  // we have to pass in the child builders to child SeqConverter in the case of
-  // converting Python objects to Arrow nested types
-  virtual Status Init(ArrayBuilder* builder) = 0;
 
-  // Append a single null value to the builder
-  virtual Status AppendNull() = 0;
-
-  // Append a valid value
-  virtual Status AppendValue(PyObject* seq) = 0;
-
-  // Append a single python object handling Null values
-  virtual Status Append(PyObject* seq) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status Extend(PyObject* seq, int64_t size) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status ExtendMasked(PyObject* seq, PyObject* mask, int64_t size) = 0;
-
-  virtual Status Close() {
-    if (chunks_.size() == 0 || builder_->length() > 0) {
-      std::shared_ptr<Array> last_chunk;
-      RETURN_NOT_OK(builder_->Finish(&last_chunk));
-      chunks_.emplace_back(std::move(last_chunk));
+  template <typename T>
+  static enable_if_string<T, Result<PyBytesView>> Convert(const T*, const O& options,
+                                                          I obj) {
+    if (options.strict) {
+      // Strict conversion, force output to be unicode / utf8 and validate that
+      // any binary values are utf8
+      ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj, true));
+      if (!view.is_utf8) {
+        return internal::InvalidValue(obj, "was not a utf8 string");
+      }
+      return std::move(view);
+    } else {
+      // Non-strict conversion; keep track of whether values are unicode or bytes
+      return PyBytesView::FromString(obj);
     }
-    return Status::OK();
   }
 
-  virtual Status GetResult(std::shared_ptr<ChunkedArray>* out) {
-    // Still some accumulated data in the builder. If there are no chunks, we
-    // always call Finish to deal with the edge case where a size-0 sequence
-    // was converted with a specific output type, like array([], type=t)
-    RETURN_NOT_OK(Close());
-    *out = std::make_shared<ChunkedArray>(this->chunks_, builder_->type());
-    return Status::OK();
+  static Result<bool> Convert(const DataType* type, const O&, I obj) {
+    return Status::NotImplemented("PyValue::Convert is not implemented for type ", type);
   }
-
-  ArrayBuilder* builder() const { return builder_; }
-
-  int num_chunks() const { return static_cast<int>(chunks_.size()); }
-
- protected:
-  ArrayBuilder* builder_;
-  bool unfinished_builder_;
-  std::vector<std::shared_ptr<Array>> chunks_;
 };
 
-template <typename Type, NullCoding null_coding = NullCoding::NONE_ONLY>
-class TypedConverter : public SeqConverter {
- public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  Status Init(ArrayBuilder* builder) override {
-    builder_ = builder;
-    DCHECK_NE(builder_, nullptr);
-    typed_builder_ = checked_cast<BuilderType*>(builder);
-    return Status::OK();
-  }
-
-  // Append a missing item (default implementation)
-  Status AppendNull() override { return this->typed_builder_->AppendNull(); }
-
-  // Append null if the obj is None or pandas null otherwise the valid value
-  Status Append(PyObject* obj) override {
-    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
-  }
-
-  Status Extend(PyObject* obj, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequence(
-        obj, [this](PyObject* item, bool* /* unused */) { return this->Append(item); });
-  }
+template <typename T>
+Status Extend(T* converter, PyObject* values, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequence(values, [converter](PyObject* item, bool* /* unused */) {
+    return converter->Append(item);
+  });
+}
 
-  Status ExtendMasked(PyObject* obj, PyObject* mask, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequenceMasked(
-        obj, mask, [this](PyObject* item, bool is_masked, bool* /* unused */) {
-          if (is_masked) {
-            return this->AppendNull();
-          } else {
-            // This will also apply the null-checking convention in the event
-            // that the value is not masked
-            return this->Append(item);  // perhaps use AppendValue instead?
-          }
-        });
-  }
+// Convert and append a sequence of values masked with a numpy array
+template <typename T>
+Status ExtendMasked(T* converter, PyObject* values, PyObject* mask, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequenceMasked(
+      values, mask, [converter](PyObject* item, bool is_masked, bool* /* unused */) {
+        if (is_masked) {
+          return converter->AppendNull();
+        } else {
+          // This will also apply the null-checking convention in the event
+          // that the value is not masked
+          return converter->Append(item);  // perhaps use AppendValue instead?
+        }
+      });
+}
 
- protected:
-  BuilderType* typed_builder_;
-};
+// The base Converter class is a mixin with predefined behavior and constructors.
+class PyConverter : public Converter<PyObject*, PyConversionOptions, PyConverter> {};
 
-// ----------------------------------------------------------------------
-// Sequence converter for null type
+template <typename T, typename Enable = void>
+class PyPrimitiveConverter;
 
-template <NullCoding null_coding>
-class NullConverter : public TypedConverter<NullType, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_null_type<T>::value || is_boolean_type<T>::value ||
+                   is_number_type<T>::value || is_decimal_type<T>::value ||
+                   is_date_type<T>::value || is_time_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  Status AppendValue(PyObject* obj) override {
-    return internal::InvalidValue(obj, "converting to null type");
-  }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter template for primitive (integer and floating point bool) types
-
-template <typename Type, NullCoding null_coding>
-class PrimitiveConverter : public TypedConverter<Type, null_coding> {
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value, ValueConverter<Type>::FromPython(obj));
-    return this->typed_builder_->Append(value);
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      return this->primitive_builder_->Append(converted);
+    }
   }
 };
 
-// ----------------------------------------------------------------------
-// Sequence converters for temporal types
-
-template <typename Type, NullCoding null_coding>
-class TimeConverter : public TypedConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_timestamp_type<T>::value || is_duration_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  explicit TimeConverter(TimeUnit::type unit, bool ignore_timezone)
-      : unit_(unit), ignore_timezone_(ignore_timezone) {}
-
-  // TODO(kszucs): support numpy values for date and time converters
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value,
-                          ValueConverter<Type>::FromPython(obj, unit_, ignore_timezone_));
-    return this->typed_builder_->Append(value);
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      // Numpy NaT sentinels can be checked after the conversion
+      if (PyArray_CheckAnyScalarExact(value) &&
+          PyValue::IsNaT(this->primitive_type_, converted)) {
+        return this->primitive_builder_->AppendNull();
+      } else {
+        return this->primitive_builder_->Append(converted);
+      }
+    }
   }
-
- protected:
-  TimeUnit::type unit_;
-  bool ignore_timezone_;
 };
 
-// TODO(kszucs): move it to the type_traits
 template <typename T>
-struct NumpyType {};
-
-template <>
-struct NumpyType<TimestampType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_DATETIME>::isnull(v);
-  }
-};
-
-template <>
-struct NumpyType<DurationType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_TIMEDELTA>::isnull(v);
-  }
-};
-
-template <typename Type, NullCoding null_coding>
-class TemporalConverter : public TimeConverter<Type, null_coding> {
+class PyPrimitiveConverter<T, enable_if_binary<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using TimeConverter<Type, null_coding>::TimeConverter;
-
-  Status AppendValue(PyObject* obj) override {
-    int64_t value;
-    if (PyArray_CheckAnyScalarExact(obj)) {
-      // convert np.datetime64 / np.timedelta64 depending on Type
-      ARROW_ASSIGN_OR_RAISE(value, ValueConverter<Type>::FromNumpy(obj, this->unit_));
-      if (NumpyType<Type>::isnull(value)) {
-        // checks numpy NaT sentinel after conversion
-        return this->typed_builder_->AppendNull();
-      }
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
     } else {
       ARROW_ASSIGN_OR_RAISE(
-          value,
-          ValueConverter<Type>::FromPython(
-              obj, this->unit_, TimeConverter<Type, null_coding>::ignore_timezone_));
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
     }
-    return this->typed_builder_->Append(value);
   }
 };
 
-// ----------------------------------------------------------------------
-// Sequence converters for Binary, FixedSizeBinary, String
-
-template <typename Type, NullCoding null_coding>
-class BinaryLikeConverter : public TypedConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_string_like<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  inline Status AutoChunk(Py_ssize_t size) {
-    // did we reach the builder size limit?
-    if (ARROW_PREDICT_FALSE(this->typed_builder_->value_data_length() + size >
-                            BuilderType::memory_limit())) {
-      // builder would be full, so need to add a new chunk
-      std::shared_ptr<Array> chunk;
-      RETURN_NOT_OK(this->typed_builder_->Finish(&chunk));
-      this->chunks_.emplace_back(std::move(chunk));
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      if (!view.is_utf8) {
+        // observed binary value
+        observed_binary_ = true;
+      }
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
     }
-    return Status::OK();
   }
 
-  Status AppendString(const PyBytesView& view) {
-    // check that the value fits in the datatype
-    if (view.size > BuilderType::memory_limit()) {
-      return Status::Invalid("string too large for datatype");
+  Result<std::shared_ptr<Array>> ToArray() override {
+    ARROW_ASSIGN_OR_RAISE(auto array, (PrimitiveConverter<T, PyConverter>::ToArray()));
+    if (observed_binary_) {
+      // if we saw any non-unicode, cast results to BinaryArray
+      auto binary_type = TypeTraits<typename T::PhysicalType>::type_singleton();
+      return array->View(binary_type);
+    } else {
+      return array;
     }
-    DCHECK_GE(view.size, 0);
-
-    // create a new chunk if the value would overflow the builder
-    RETURN_NOT_OK(AutoChunk(view.size));
-
-    // now we can safely append the value to the builder
-    RETURN_NOT_OK(
-        this->typed_builder_->Append(::arrow::util::string_view(view.bytes, view.size)));
-
-    return Status::OK();
   }
 
  protected:
-  // Create a single instance of PyBytesView here to prevent unnecessary object
-  // creation/destruction
-  PyBytesView string_view_;
+  bool observed_binary_ = false;
 };
 
-template <typename Type, NullCoding null_coding>
-class BinaryConverter : public BinaryLikeConverter<Type, null_coding> {
- public:
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto view, ValueConverter<Type>::FromPython(obj));
-    return this->AppendString(view);
-  }
-};
+template <typename U, typename Enable = void>
+class PyDictionaryConverter;
 
-template <NullCoding null_coding>
-class FixedSizeBinaryConverter
-    : public BinaryLikeConverter<FixedSizeBinaryType, null_coding> {
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_c_type<U>>
+    : public DictionaryConverter<U, PyConverter> {
  public:
-  explicit FixedSizeBinaryConverter(int32_t byte_width) : byte_width_(byte_width) {}
-
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(
-        this->string_view_,
-        ValueConverter<FixedSizeBinaryType>::FromPython(obj, byte_width_));
-    return this->AppendString(this->string_view_);
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(auto converted,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(converted);
+    }
   }
-
- protected:
-  int32_t byte_width_;
 };
 
-// For String/UTF8, if strict_conversions enabled, we reject any non-UTF8,
-// otherwise we allow but return results as BinaryArray
-template <typename Type, bool Strict, NullCoding null_coding>
-class StringConverter : public BinaryLikeConverter<Type, null_coding> {
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_string_view<U>>
+    : public DictionaryConverter<U, PyConverter> {
  public:
-  StringConverter() : binary_count_(0) {}
-
-  Status AppendValue(PyObject* obj) override {
-    if (Strict) {
-      // raise if the object is not unicode or not an utf-8 encoded bytes
-      ARROW_ASSIGN_OR_RAISE(this->string_view_, ValueConverter<Type>::FromPython(obj));
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
     } else {
-      // keep track of whether values are unicode or bytes; if any bytes are
-      // observe, the result will be bytes
-      bool is_utf8;
-      ARROW_ASSIGN_OR_RAISE(this->string_view_,
-                            ValueConverter<Type>::FromPython(obj, &is_utf8));
-      if (!is_utf8) {
-        ++binary_count_;
-      }
+      ARROW_ASSIGN_OR_RAISE(auto view,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(util::string_view(view.bytes, view.size));
     }
-    return this->AppendString(this->string_view_);
   }
-
-  Status GetResult(std::shared_ptr<ChunkedArray>* out) override {
-    RETURN_NOT_OK(SeqConverter::GetResult(out));
-
-    // If we saw any non-unicode, cast results to BinaryArray
-    if (binary_count_) {
-      // We should have bailed out earlier
-      DCHECK(!Strict);
-      auto binary_type = TypeTraits<typename Type::PhysicalType>::type_singleton();
-      return (*out)->View(binary_type).Value(out);
-    }
-    return Status::OK();
-  }
-
- protected:
-  int64_t binary_count_;
 };
 
-// ----------------------------------------------------------------------
-// Convert lists (NumPy arrays containing lists or ndarrays as values)
-
 // If the value type does not match the expected NumPy dtype, then fall through
 // to a slower PySequence-based path
-#define LIST_FAST_CASE(TYPE, NUMPY_TYPE, ArrowType)            \
-  case Type::TYPE: {                                           \
-    if (PyArray_DESCR(arr)->type_num != NUMPY_TYPE) {          \
-      return value_converter_->Extend(obj, value_length);      \
-    }                                                          \
-    return AppendNdarrayTypedItem<NUMPY_TYPE, ArrowType>(arr); \
+#define LIST_FAST_CASE(TYPE_ID, TYPE, NUMPY_TYPE)               \
+  case Type::TYPE_ID: {                                         \
+    if (PyArray_DESCR(ndarray)->type_num != NUMPY_TYPE) {       \
+      return Extend(this->value_converter_.get(), value, size); \
+    }                                                           \
+    return AppendNdarrayTyped<TYPE, NUMPY_TYPE>(ndarray);       \
   }
 
 // Use internal::VisitSequence, fast for NPY_OBJECT but slower otherwise
-#define LIST_SLOW_CASE(TYPE)                            \
-  case Type::TYPE: {                                    \
-    return value_converter_->Extend(obj, value_length); \
+#define LIST_SLOW_CASE(TYPE_ID)                               \
+  case Type::TYPE_ID: {                                       \
+    return Extend(this->value_converter_.get(), value, size); \
   }
 
-// Base class for ListConverter and FixedSizeListConverter (to have both work with CRTP)
-template <typename TypeClass, NullCoding null_coding>
-class BaseListConverter : public TypedConverter<TypeClass, null_coding> {
+template <typename T>
+class PyListConverter : public ListConverter<T, PyConverter> {
  public:
-  using BuilderType = typename TypeTraits<TypeClass>::BuilderType;
-
-  explicit BaseListConverter(bool from_pandas, bool strict_conversions,
-                             bool ignore_timezone)
-      : from_pandas_(from_pandas),
-        strict_conversions_(strict_conversions),
-        ignore_timezone_(ignore_timezone) {}
-
-  Status Init(ArrayBuilder* builder) override {
-    this->builder_ = builder;
-    this->typed_builder_ = checked_cast<BuilderType*>(builder);
-
-    this->value_type_ = checked_cast<const TypeClass&>(*builder->type()).value_type();
-    RETURN_NOT_OK(GetConverter(value_type_, from_pandas_, strict_conversions_,
-                               ignore_timezone_, &value_converter_));
-    return this->value_converter_->Init(this->typed_builder_->value_builder());
+  Status ValidateOverflow(const MapType*, int64_t size) { return Status::OK(); }
+
+  Status ValidateOverflow(const BaseListType*, int64_t size) {
+    return this->list_builder_->ValidateOverflow(size);
   }
 
-  template <int NUMPY_TYPE, typename Type>
-  Status AppendNdarrayTypedItem(PyArrayObject* arr) {
-    using traits = internal::npy_traits<NUMPY_TYPE>;
-    using T = typename traits::value_type;
-    using ValueBuilderType = typename TypeTraits<Type>::BuilderType;
+  Status ValidateBuilder(const MapType*) {
+    if (this->list_builder_->key_builder()->null_count() > 0) {
+      return Status::Invalid("Invalid Map: key field can not contain null values");
+    } else {
+      return Status::OK();
+    }
+  }
 
-    const bool null_sentinels_possible =
-        // Always treat Numpy's NaT as null
-        NUMPY_TYPE == NPY_DATETIME || NUMPY_TYPE == NPY_TIMEDELTA ||
-        // Observing pandas's null sentinels
-        (from_pandas_ && traits::supports_nulls);
+  Status ValidateBuilder(const BaseListType*) { return Status::OK(); }
 
-    auto child_builder = checked_cast<ValueBuilderType*>(value_converter_->builder());
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->list_builder_->AppendNull();
+    }
 
-    // TODO(wesm): Vector append when not strided
-    Ndarray1DIndexer<T> values(arr);
-    if (null_sentinels_possible) {
-      for (int64_t i = 0; i < values.size(); ++i) {
-        if (traits::isnull(values[i])) {
-          RETURN_NOT_OK(child_builder->AppendNull());
-        } else {
-          RETURN_NOT_OK(child_builder->Append(values[i]));
-        }
-      }
+    RETURN_NOT_OK(this->list_builder_->Append());
+    if (PyArray_Check(value)) {
+      RETURN_NOT_OK(AppendNdarray(value));
+    } else if (PySequence_Check(value)) {
+      RETURN_NOT_OK(AppendSequence(value));
     } else {
-      for (int64_t i = 0; i < values.size(); ++i) {
-        RETURN_NOT_OK(child_builder->Append(values[i]));
-      }
+      return internal::InvalidType(
+          value, "was not a sequence or recognized null for conversion to list type");
     }
-    return Status::OK();
+
+    return ValidateBuilder(this->list_type_);
   }
 
-  Status AppendNdarrayItem(PyObject* obj) {
-    PyArrayObject* arr = reinterpret_cast<PyArrayObject*>(obj);
+  Status AppendSequence(PyObject* value) {
+    int64_t size = static_cast<int64_t>(PySequence_Size(value));
+    RETURN_NOT_OK(ValidateOverflow(this->list_type_, size));
+    return Extend(this->value_converter_.get(), value, size);
+  }
 
-    if (PyArray_NDIM(arr) != 1) {
+  Status AppendNdarray(PyObject* value) {
+    PyArrayObject* ndarray = reinterpret_cast<PyArrayObject*>(value);
+    if (PyArray_NDIM(ndarray) != 1) {
       return Status::Invalid("Can only convert 1-dimensional array values");
     }
+    const int64_t size = PyArray_SIZE(ndarray);
+    RETURN_NOT_OK(ValidateOverflow(this->list_type_, size));
 
-    const int64_t value_length = PyArray_SIZE(arr);
-
-    switch (value_type_->id()) {
+    const auto value_type = this->value_converter_->builder()->type();
+    switch (value_type->id()) {
       LIST_SLOW_CASE(NA)
-      LIST_FAST_CASE(UINT8, NPY_UINT8, UInt8Type)
-      LIST_FAST_CASE(INT8, NPY_INT8, Int8Type)
-      LIST_FAST_CASE(UINT16, NPY_UINT16, UInt16Type)
-      LIST_FAST_CASE(INT16, NPY_INT16, Int16Type)
-      LIST_FAST_CASE(UINT32, NPY_UINT32, UInt32Type)
-      LIST_FAST_CASE(INT32, NPY_INT32, Int32Type)
-      LIST_FAST_CASE(UINT64, NPY_UINT64, UInt64Type)
-      LIST_FAST_CASE(INT64, NPY_INT64, Int64Type)
+      LIST_FAST_CASE(UINT8, UInt8Type, NPY_UINT8)
+      LIST_FAST_CASE(INT8, Int8Type, NPY_INT8)
+      LIST_FAST_CASE(UINT16, UInt16Type, NPY_UINT16)
+      LIST_FAST_CASE(INT16, Int16Type, NPY_INT16)
+      LIST_FAST_CASE(UINT32, UInt32Type, NPY_UINT32)
+      LIST_FAST_CASE(INT32, Int32Type, NPY_INT32)
+      LIST_FAST_CASE(UINT64, UInt64Type, NPY_UINT64)
+      LIST_FAST_CASE(INT64, Int64Type, NPY_INT64)
+      LIST_FAST_CASE(HALF_FLOAT, HalfFloatType, NPY_FLOAT16)
+      LIST_FAST_CASE(FLOAT, FloatType, NPY_FLOAT)
+      LIST_FAST_CASE(DOUBLE, DoubleType, NPY_DOUBLE)
+      LIST_FAST_CASE(TIMESTAMP, TimestampType, NPY_DATETIME)
+      LIST_FAST_CASE(DURATION, DurationType, NPY_TIMEDELTA)
       LIST_SLOW_CASE(DATE32)
       LIST_SLOW_CASE(DATE64)
       LIST_SLOW_CASE(TIME32)
       LIST_SLOW_CASE(TIME64)
-      LIST_FAST_CASE(TIMESTAMP, NPY_DATETIME, TimestampType)
-      LIST_FAST_CASE(DURATION, NPY_TIMEDELTA, DurationType)
-      LIST_FAST_CASE(HALF_FLOAT, NPY_FLOAT16, HalfFloatType)
-      LIST_FAST_CASE(FLOAT, NPY_FLOAT, FloatType)
-      LIST_FAST_CASE(DOUBLE, NPY_DOUBLE, DoubleType)
       LIST_SLOW_CASE(BINARY)
       LIST_SLOW_CASE(FIXED_SIZE_BINARY)
       LIST_SLOW_CASE(STRING)
       case Type::LIST: {
-        if (PyArray_DESCR(arr)->type_num != NPY_OBJECT) {
+        if (PyArray_DESCR(ndarray)->type_num != NPY_OBJECT) {
           return Status::Invalid(
-              "Can only convert list types from NumPy object "
-              "array input");
+              "Can only convert list types from NumPy object array input");
         }
-        return internal::VisitSequence(obj, [this](PyObject* item, bool*) {
-          return value_converter_->Append(item);
+        return internal::VisitSequence(value, [this](PyObject* item, bool*) {
+          return this->value_converter_->Append(item);
         });

Review comment:
       Updated.

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,
+                            std::shared_ptr<ArrayBuilder> builder,
+                            const std::vector<std::shared_ptr<Self>>& children,

Review comment:
       Updated.

##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -329,985 +302,602 @@ struct ValueConverter<DurationType> {
         default:
           return Status::UnknownError("Invalid time unit");
       }
+    } else if (PyArray_CheckAnyScalarExact(obj)) {
+      // validate that the numpy scalar has np.datetime64 dtype
+      std::shared_ptr<DataType> numpy_type;
+      RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &numpy_type));
+      if (!numpy_type->Equals(*type)) {
+        return Status::NotImplemented("Expected np.timedelta64 but got: ",
+                                      numpy_type->ToString());
+      }
+      return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
     } else {
       RETURN_NOT_OK(internal::CIntFromPython(obj, &value));
     }
     return value;
   }
 
-  static inline Result<int64_t> FromNumpy(PyObject* obj, TimeUnit::type unit) {
-    // validate that the numpy scalar has np.timedelta64 dtype
-    std::shared_ptr<DataType> type;
-    RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &type));
-    if (type->id() != DurationType::type_id) {
-      // TODO(kszucs): the message should highlight the received numpy dtype
-      return Status::Invalid("Expected np.timedelta64 but got: ", type->ToString());
-    }
-    // validate that the time units are matching
-    if (unit != checked_cast<const DurationType&>(*type).unit()) {
-      return Status::NotImplemented(
-          "Cannot convert NumPy np.timedelta64 objects with differing unit");
-    }
-    // convert the numpy value
-    return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_any_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    return std::move(view);
-  }
-};
+  // The binary-like intermediate representation is PyBytesView because it keeps temporary
+  // python objects alive (non-contiguous memoryview) and stores whether the original
+  // object was unicode encoded or not, which is used for unicode -> bytes coersion if
+  // there is a non-unicode object observed.
 
-template <typename Type>
-struct ValueConverter<Type, enable_if_string_like<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    // strict conversion, force output to be unicode / utf8 and validate that
-    // any binary values are utf8
-    bool is_utf8 = false;
-    PyBytesView view;
-
-    RETURN_NOT_OK(view.FromString(obj, &is_utf8));
-    if (!is_utf8) {
-      return internal::InvalidValue(obj, "was not a utf8 string");
-    }
-    return std::move(view);
+  static Result<PyBytesView> Convert(const BaseBinaryType*, const O&, I obj) {
+    return PyBytesView::FromString(obj);
   }
 
-  static inline Result<PyBytesView> FromPython(PyObject* obj, bool* is_utf8) {
-    PyBytesView view;
-
-    // Non-strict conversion; keep track of whether values are unicode or bytes
-    if (PyUnicode_Check(obj)) {
-      *is_utf8 = true;
-      RETURN_NOT_OK(view.FromUnicode(obj));
+  static Result<PyBytesView> Convert(const FixedSizeBinaryType* type, const O&, I obj) {
+    ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj));
+    if (ARROW_PREDICT_TRUE(view.size == type->byte_width())) {
+      return std::move(view);
     } else {
-      // If not unicode or bytes, FromBinary will error
-      *is_utf8 = false;
-      RETURN_NOT_OK(view.FromBinary(obj));
-    }
-    return std::move(view);
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_fixed_size_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj, int32_t byte_width) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    if (ARROW_PREDICT_FALSE(view.size != byte_width)) {
       std::stringstream ss;
-      ss << "expected to be length " << byte_width << " was " << view.size;
+      ss << "expected to be length " << type->byte_width() << " was " << view.size;
       return internal::InvalidValue(obj, ss.str());
-    } else {
-      return std::move(view);
     }
   }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter base and CRTP "middle" subclasses
-
-class SeqConverter;
-
-// Forward-declare converter factory
-Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
-                    bool strict_conversions, bool ignore_timezone,
-                    std::unique_ptr<SeqConverter>* out);
-
-// Marshal Python sequence (list, tuple, etc.) to Arrow array
-class SeqConverter {
- public:
-  virtual ~SeqConverter() = default;
-
-  // Initialize the sequence converter with an ArrayBuilder created
-  // externally. The reason for this interface is that we have
-  // arrow::MakeBuilder which also creates child builders for nested types, so
-  // we have to pass in the child builders to child SeqConverter in the case of
-  // converting Python objects to Arrow nested types
-  virtual Status Init(ArrayBuilder* builder) = 0;
-
-  // Append a single null value to the builder
-  virtual Status AppendNull() = 0;
-
-  // Append a valid value
-  virtual Status AppendValue(PyObject* seq) = 0;
-
-  // Append a single python object handling Null values
-  virtual Status Append(PyObject* seq) = 0;
 
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status Extend(PyObject* seq, int64_t size) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status ExtendMasked(PyObject* seq, PyObject* mask, int64_t size) = 0;
-
-  virtual Status Close() {
-    if (chunks_.size() == 0 || builder_->length() > 0) {
-      std::shared_ptr<Array> last_chunk;
-      RETURN_NOT_OK(builder_->Finish(&last_chunk));
-      chunks_.emplace_back(std::move(last_chunk));
+  template <typename T>
+  static enable_if_string<T, Result<PyBytesView>> Convert(const T*, const O& options,
+                                                          I obj) {
+    if (options.strict) {
+      // Strict conversion, force output to be unicode / utf8 and validate that
+      // any binary values are utf8
+      ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj, true));
+      if (!view.is_utf8) {
+        return internal::InvalidValue(obj, "was not a utf8 string");
+      }
+      return std::move(view);
+    } else {
+      // Non-strict conversion; keep track of whether values are unicode or bytes
+      return PyBytesView::FromString(obj);
     }
-    return Status::OK();
   }
 
-  virtual Status GetResult(std::shared_ptr<ChunkedArray>* out) {
-    // Still some accumulated data in the builder. If there are no chunks, we
-    // always call Finish to deal with the edge case where a size-0 sequence
-    // was converted with a specific output type, like array([], type=t)
-    RETURN_NOT_OK(Close());
-    *out = std::make_shared<ChunkedArray>(this->chunks_, builder_->type());
-    return Status::OK();
+  static Result<bool> Convert(const DataType* type, const O&, I obj) {
+    return Status::NotImplemented("PyValue::Convert is not implemented for type ", type);
   }
-
-  ArrayBuilder* builder() const { return builder_; }
-
-  int num_chunks() const { return static_cast<int>(chunks_.size()); }
-
- protected:
-  ArrayBuilder* builder_;
-  bool unfinished_builder_;
-  std::vector<std::shared_ptr<Array>> chunks_;
 };
 
-template <typename Type, NullCoding null_coding = NullCoding::NONE_ONLY>
-class TypedConverter : public SeqConverter {
- public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  Status Init(ArrayBuilder* builder) override {
-    builder_ = builder;
-    DCHECK_NE(builder_, nullptr);
-    typed_builder_ = checked_cast<BuilderType*>(builder);
-    return Status::OK();
-  }
-
-  // Append a missing item (default implementation)
-  Status AppendNull() override { return this->typed_builder_->AppendNull(); }
-
-  // Append null if the obj is None or pandas null otherwise the valid value
-  Status Append(PyObject* obj) override {
-    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
-  }
-
-  Status Extend(PyObject* obj, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequence(
-        obj, [this](PyObject* item, bool* /* unused */) { return this->Append(item); });
-  }
-
-  Status ExtendMasked(PyObject* obj, PyObject* mask, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequenceMasked(
-        obj, mask, [this](PyObject* item, bool is_masked, bool* /* unused */) {
-          if (is_masked) {
-            return this->AppendNull();
-          } else {
-            // This will also apply the null-checking convention in the event
-            // that the value is not masked
-            return this->Append(item);  // perhaps use AppendValue instead?
-          }
-        });
-  }
+template <typename T>
+Status Extend(T* converter, PyObject* values, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequence(values, [converter](PyObject* item, bool* /* unused */) {
+    return converter->Append(item);
+  });
+}
 
- protected:
-  BuilderType* typed_builder_;
-};
+// Convert and append a sequence of values masked with a numpy array
+template <typename T>
+Status ExtendMasked(T* converter, PyObject* values, PyObject* mask, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequenceMasked(
+      values, mask, [converter](PyObject* item, bool is_masked, bool* /* unused */) {
+        if (is_masked) {
+          return converter->AppendNull();
+        } else {
+          // This will also apply the null-checking convention in the event
+          // that the value is not masked
+          return converter->Append(item);  // perhaps use AppendValue instead?
+        }
+      });
+}
 
-// ----------------------------------------------------------------------
-// Sequence converter for null type
+// The base Converter class is a mixin with predefined behavior and constructors.
+using PyConverter = Converter<PyObject*, PyConversionOptions>;
 
-template <NullCoding null_coding>
-class NullConverter : public TypedConverter<NullType, null_coding> {
- public:
-  Status AppendValue(PyObject* obj) override {
-    return internal::InvalidValue(obj, "converting to null type");
-  }
-};
+template <typename T, typename Enable = void>
+class PyPrimitiveConverter;
 
-// ----------------------------------------------------------------------
-// Sequence converter template for primitive (integer and floating point bool) types
+template <typename T>
+class PyListConverter;
 
-template <typename Type, NullCoding null_coding>
-class PrimitiveConverter : public TypedConverter<Type, null_coding> {
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value, ValueConverter<Type>::FromPython(obj));
-    return this->typed_builder_->Append(value);
-  }
-};
+template <typename U, typename Enable = void>
+class PyDictionaryConverter;
 
-// ----------------------------------------------------------------------
-// Sequence converters for temporal types
+class PyStructConverter;
 
-template <typename Type, NullCoding null_coding>
-class TimeConverter : public TypedConverter<Type, null_coding> {
- public:
-  explicit TimeConverter(TimeUnit::type unit, bool ignore_timezone)
-      : unit_(unit), ignore_timezone_(ignore_timezone) {}
-
-  // TODO(kszucs): support numpy values for date and time converters
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value,
-                          ValueConverter<Type>::FromPython(obj, unit_, ignore_timezone_));
-    return this->typed_builder_->Append(value);
-  }
+template <typename T, typename Enable = void>
+struct PyConverterTrait;
 
- protected:
-  TimeUnit::type unit_;
-  bool ignore_timezone_;
+template <typename T>
+struct PyConverterTrait<
+    T, enable_if_t<!is_nested_type<T>::value && !is_interval_type<T>::value &&
+                   !is_extension_type<T>::value>> {
+  using type = PyPrimitiveConverter<T>;
 };
 
-// TODO(kszucs): move it to the type_traits
 template <typename T>
-struct NumpyType {};
+struct PyConverterTrait<T, enable_if_list_like<T>> {
+  using type = PyListConverter<T>;
+};
 
 template <>
-struct NumpyType<TimestampType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_DATETIME>::isnull(v);
-  }
+struct PyConverterTrait<StructType> {
+  using type = PyStructConverter;
 };
 
 template <>
-struct NumpyType<DurationType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_TIMEDELTA>::isnull(v);
-  }
+struct PyConverterTrait<DictionaryType> {
+  template <typename T>
+  using type = PyDictionaryConverter<T>;
 };
 
-template <typename Type, NullCoding null_coding>
-class TemporalConverter : public TimeConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_null_type<T>::value || is_boolean_type<T>::value ||
+                   is_number_type<T>::value || is_decimal_type<T>::value ||
+                   is_date_type<T>::value || is_time_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using TimeConverter<Type, null_coding>::TimeConverter;
-
-  Status AppendValue(PyObject* obj) override {
-    int64_t value;
-    if (PyArray_CheckAnyScalarExact(obj)) {
-      // convert np.datetime64 / np.timedelta64 depending on Type
-      ARROW_ASSIGN_OR_RAISE(value, ValueConverter<Type>::FromNumpy(obj, this->unit_));
-      if (NumpyType<Type>::isnull(value)) {
-        // checks numpy NaT sentinel after conversion
-        return this->typed_builder_->AppendNull();
-      }
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
     } else {
       ARROW_ASSIGN_OR_RAISE(
-          value,
-          ValueConverter<Type>::FromPython(
-              obj, this->unit_, TimeConverter<Type, null_coding>::ignore_timezone_));
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      return this->primitive_builder_->Append(converted);
     }
-    return this->typed_builder_->Append(value);
   }
 };
 
-// ----------------------------------------------------------------------
-// Sequence converters for Binary, FixedSizeBinary, String
-
-template <typename Type, NullCoding null_coding>
-class BinaryLikeConverter : public TypedConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_timestamp_type<T>::value || is_duration_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  inline Status AutoChunk(Py_ssize_t size) {
-    // did we reach the builder size limit?
-    if (ARROW_PREDICT_FALSE(this->typed_builder_->value_data_length() + size >
-                            BuilderType::memory_limit())) {
-      // builder would be full, so need to add a new chunk
-      std::shared_ptr<Array> chunk;
-      RETURN_NOT_OK(this->typed_builder_->Finish(&chunk));
-      this->chunks_.emplace_back(std::move(chunk));
-    }
-    return Status::OK();
-  }
-
-  Status AppendString(const PyBytesView& view) {
-    // check that the value fits in the datatype
-    if (view.size > BuilderType::memory_limit()) {
-      return Status::Invalid("string too large for datatype");
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      // Numpy NaT sentinels can be checked after the conversion
+      if (PyArray_CheckAnyScalarExact(value) &&
+          PyValue::IsNaT(this->primitive_type_, converted)) {
+        return this->primitive_builder_->AppendNull();
+      } else {
+        return this->primitive_builder_->Append(converted);
+      }
     }
-    DCHECK_GE(view.size, 0);
-
-    // create a new chunk if the value would overflow the builder
-    RETURN_NOT_OK(AutoChunk(view.size));
-
-    // now we can safely append the value to the builder
-    RETURN_NOT_OK(
-        this->typed_builder_->Append(::arrow::util::string_view(view.bytes, view.size)));
-
-    return Status::OK();
   }
-
- protected:
-  // Create a single instance of PyBytesView here to prevent unnecessary object
-  // creation/destruction
-  PyBytesView string_view_;
 };
 
-template <typename Type, NullCoding null_coding>
-class BinaryConverter : public BinaryLikeConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_binary<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto view, ValueConverter<Type>::FromPython(obj));
-    return this->AppendString(view);
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
+    }
   }
 };
 
-template <NullCoding null_coding>
-class FixedSizeBinaryConverter
-    : public BinaryLikeConverter<FixedSizeBinaryType, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_string_like<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  explicit FixedSizeBinaryConverter(int32_t byte_width) : byte_width_(byte_width) {}
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      if (!view.is_utf8) {
+        // observed binary value
+        observed_binary_ = true;
+      }
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
+    }
+  }
 
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(
-        this->string_view_,
-        ValueConverter<FixedSizeBinaryType>::FromPython(obj, byte_width_));
-    return this->AppendString(this->string_view_);
+  Result<std::shared_ptr<Array>> ToArray() override {
+    ARROW_ASSIGN_OR_RAISE(auto array, (PrimitiveConverter<T, PyConverter>::ToArray()));
+    if (observed_binary_) {
+      // if we saw any non-unicode, cast results to BinaryArray
+      auto binary_type = TypeTraits<typename T::PhysicalType>::type_singleton();
+      return array->View(binary_type);
+    } else {
+      return array;
+    }
   }
 
  protected:
-  int32_t byte_width_;
+  bool observed_binary_ = false;
 };
 
-// For String/UTF8, if strict_conversions enabled, we reject any non-UTF8,
-// otherwise we allow but return results as BinaryArray
-template <typename Type, bool Strict, NullCoding null_coding>
-class StringConverter : public BinaryLikeConverter<Type, null_coding> {
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_c_type<U>>
+    : public DictionaryConverter<U, PyConverter> {
  public:
-  StringConverter() : binary_count_(0) {}
-
-  Status AppendValue(PyObject* obj) override {
-    if (Strict) {
-      // raise if the object is not unicode or not an utf-8 encoded bytes
-      ARROW_ASSIGN_OR_RAISE(this->string_view_, ValueConverter<Type>::FromPython(obj));
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
     } else {
-      // keep track of whether values are unicode or bytes; if any bytes are
-      // observe, the result will be bytes
-      bool is_utf8;
-      ARROW_ASSIGN_OR_RAISE(this->string_view_,
-                            ValueConverter<Type>::FromPython(obj, &is_utf8));
-      if (!is_utf8) {
-        ++binary_count_;
-      }
+      ARROW_ASSIGN_OR_RAISE(auto converted,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(converted);
     }
-    return this->AppendString(this->string_view_);
   }
+};
 
-  Status GetResult(std::shared_ptr<ChunkedArray>* out) override {
-    RETURN_NOT_OK(SeqConverter::GetResult(out));
-
-    // If we saw any non-unicode, cast results to BinaryArray
-    if (binary_count_) {
-      // We should have bailed out earlier
-      DCHECK(!Strict);
-      auto binary_type = TypeTraits<typename Type::PhysicalType>::type_singleton();
-      return (*out)->View(binary_type).Value(out);
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_string_view<U>>
+    : public DictionaryConverter<U, PyConverter> {
+ public:
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(auto view,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(util::string_view(view.bytes, view.size));
     }
-    return Status::OK();
   }
-
- protected:
-  int64_t binary_count_;
 };
 
-// ----------------------------------------------------------------------
-// Convert lists (NumPy arrays containing lists or ndarrays as values)
-
 // If the value type does not match the expected NumPy dtype, then fall through
 // to a slower PySequence-based path
-#define LIST_FAST_CASE(TYPE, NUMPY_TYPE, ArrowType)            \
-  case Type::TYPE: {                                           \
-    if (PyArray_DESCR(arr)->type_num != NUMPY_TYPE) {          \
-      return value_converter_->Extend(obj, value_length);      \
-    }                                                          \
-    return AppendNdarrayTypedItem<NUMPY_TYPE, ArrowType>(arr); \
+#define LIST_FAST_CASE(TYPE_ID, TYPE, NUMPY_TYPE)               \
+  case Type::TYPE_ID: {                                         \
+    if (PyArray_DESCR(ndarray)->type_num != NUMPY_TYPE) {       \
+      return Extend(this->value_converter_.get(), value, size); \
+    }                                                           \
+    return AppendNdarrayTyped<TYPE, NUMPY_TYPE>(ndarray);       \
   }
 
 // Use internal::VisitSequence, fast for NPY_OBJECT but slower otherwise
-#define LIST_SLOW_CASE(TYPE)                            \
-  case Type::TYPE: {                                    \
-    return value_converter_->Extend(obj, value_length); \
+#define LIST_SLOW_CASE(TYPE_ID)                               \
+  case Type::TYPE_ID: {                                       \
+    return Extend(this->value_converter_.get(), value, size); \
   }
 
-// Base class for ListConverter and FixedSizeListConverter (to have both work with CRTP)
-template <typename TypeClass, NullCoding null_coding>
-class BaseListConverter : public TypedConverter<TypeClass, null_coding> {
+template <typename T>
+class PyListConverter : public ListConverter<T, PyConverter, PyConverterTrait> {
  public:
-  using BuilderType = typename TypeTraits<TypeClass>::BuilderType;
-
-  explicit BaseListConverter(bool from_pandas, bool strict_conversions,
-                             bool ignore_timezone)
-      : from_pandas_(from_pandas),
-        strict_conversions_(strict_conversions),
-        ignore_timezone_(ignore_timezone) {}
-
-  Status Init(ArrayBuilder* builder) override {
-    this->builder_ = builder;
-    this->typed_builder_ = checked_cast<BuilderType*>(builder);
-
-    this->value_type_ = checked_cast<const TypeClass&>(*builder->type()).value_type();
-    RETURN_NOT_OK(GetConverter(value_type_, from_pandas_, strict_conversions_,
-                               ignore_timezone_, &value_converter_));
-    return this->value_converter_->Init(this->typed_builder_->value_builder());
-  }
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->list_builder_->AppendNull();
+    }
 
-  template <int NUMPY_TYPE, typename Type>
-  Status AppendNdarrayTypedItem(PyArrayObject* arr) {
-    using traits = internal::npy_traits<NUMPY_TYPE>;
-    using T = typename traits::value_type;
-    using ValueBuilderType = typename TypeTraits<Type>::BuilderType;
+    RETURN_NOT_OK(this->list_builder_->Append());
+    if (PyArray_Check(value)) {
+      RETURN_NOT_OK(AppendNdarray(value));
+    } else if (PySequence_Check(value)) {
+      RETURN_NOT_OK(AppendSequence(value));
+    } else {
+      return internal::InvalidType(
+          value, "was not a sequence or recognized null for conversion to list type");
+    }
 
-    const bool null_sentinels_possible =
-        // Always treat Numpy's NaT as null
-        NUMPY_TYPE == NPY_DATETIME || NUMPY_TYPE == NPY_TIMEDELTA ||
-        // Observing pandas's null sentinels
-        (from_pandas_ && traits::supports_nulls);
+    return ValidateBuilder(this->list_type_);
+  }
 
-    auto child_builder = checked_cast<ValueBuilderType*>(value_converter_->builder());
+ protected:
+  Status ValidateOverflow(const MapType*, int64_t size) { return Status::OK(); }

Review comment:
       Also need to write a test case for 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.

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -329,985 +302,602 @@ struct ValueConverter<DurationType> {
         default:
           return Status::UnknownError("Invalid time unit");
       }
+    } else if (PyArray_CheckAnyScalarExact(obj)) {
+      // validate that the numpy scalar has np.datetime64 dtype
+      std::shared_ptr<DataType> numpy_type;
+      RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &numpy_type));
+      if (!numpy_type->Equals(*type)) {
+        return Status::NotImplemented("Expected np.timedelta64 but got: ",
+                                      numpy_type->ToString());
+      }
+      return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
     } else {
       RETURN_NOT_OK(internal::CIntFromPython(obj, &value));
     }
     return value;
   }
 
-  static inline Result<int64_t> FromNumpy(PyObject* obj, TimeUnit::type unit) {
-    // validate that the numpy scalar has np.timedelta64 dtype
-    std::shared_ptr<DataType> type;
-    RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &type));
-    if (type->id() != DurationType::type_id) {
-      // TODO(kszucs): the message should highlight the received numpy dtype
-      return Status::Invalid("Expected np.timedelta64 but got: ", type->ToString());
-    }
-    // validate that the time units are matching
-    if (unit != checked_cast<const DurationType&>(*type).unit()) {
-      return Status::NotImplemented(
-          "Cannot convert NumPy np.timedelta64 objects with differing unit");
-    }
-    // convert the numpy value
-    return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_any_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    return std::move(view);
-  }
-};
+  // The binary-like intermediate representation is PyBytesView because it keeps temporary
+  // python objects alive (non-contiguous memoryview) and stores whether the original
+  // object was unicode encoded or not, which is used for unicode -> bytes coersion if
+  // there is a non-unicode object observed.
 
-template <typename Type>
-struct ValueConverter<Type, enable_if_string_like<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    // strict conversion, force output to be unicode / utf8 and validate that
-    // any binary values are utf8
-    bool is_utf8 = false;
-    PyBytesView view;
-
-    RETURN_NOT_OK(view.FromString(obj, &is_utf8));
-    if (!is_utf8) {
-      return internal::InvalidValue(obj, "was not a utf8 string");
-    }
-    return std::move(view);
+  static Result<PyBytesView> Convert(const BaseBinaryType*, const O&, I obj) {
+    return PyBytesView::FromString(obj);
   }
 
-  static inline Result<PyBytesView> FromPython(PyObject* obj, bool* is_utf8) {
-    PyBytesView view;
-
-    // Non-strict conversion; keep track of whether values are unicode or bytes
-    if (PyUnicode_Check(obj)) {
-      *is_utf8 = true;
-      RETURN_NOT_OK(view.FromUnicode(obj));
+  static Result<PyBytesView> Convert(const FixedSizeBinaryType* type, const O&, I obj) {
+    ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj));
+    if (ARROW_PREDICT_TRUE(view.size == type->byte_width())) {
+      return std::move(view);
     } else {
-      // If not unicode or bytes, FromBinary will error
-      *is_utf8 = false;
-      RETURN_NOT_OK(view.FromBinary(obj));
-    }
-    return std::move(view);
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_fixed_size_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj, int32_t byte_width) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    if (ARROW_PREDICT_FALSE(view.size != byte_width)) {
       std::stringstream ss;
-      ss << "expected to be length " << byte_width << " was " << view.size;
+      ss << "expected to be length " << type->byte_width() << " was " << view.size;
       return internal::InvalidValue(obj, ss.str());
-    } else {
-      return std::move(view);
     }
   }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter base and CRTP "middle" subclasses
-
-class SeqConverter;
-
-// Forward-declare converter factory
-Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
-                    bool strict_conversions, bool ignore_timezone,
-                    std::unique_ptr<SeqConverter>* out);
-
-// Marshal Python sequence (list, tuple, etc.) to Arrow array
-class SeqConverter {
- public:
-  virtual ~SeqConverter() = default;
-
-  // Initialize the sequence converter with an ArrayBuilder created
-  // externally. The reason for this interface is that we have
-  // arrow::MakeBuilder which also creates child builders for nested types, so
-  // we have to pass in the child builders to child SeqConverter in the case of
-  // converting Python objects to Arrow nested types
-  virtual Status Init(ArrayBuilder* builder) = 0;
-
-  // Append a single null value to the builder
-  virtual Status AppendNull() = 0;
-
-  // Append a valid value
-  virtual Status AppendValue(PyObject* seq) = 0;
-
-  // Append a single python object handling Null values
-  virtual Status Append(PyObject* seq) = 0;
 
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status Extend(PyObject* seq, int64_t size) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status ExtendMasked(PyObject* seq, PyObject* mask, int64_t size) = 0;
-
-  virtual Status Close() {
-    if (chunks_.size() == 0 || builder_->length() > 0) {
-      std::shared_ptr<Array> last_chunk;
-      RETURN_NOT_OK(builder_->Finish(&last_chunk));
-      chunks_.emplace_back(std::move(last_chunk));
+  template <typename T>
+  static enable_if_string<T, Result<PyBytesView>> Convert(const T*, const O& options,
+                                                          I obj) {
+    if (options.strict) {
+      // Strict conversion, force output to be unicode / utf8 and validate that
+      // any binary values are utf8
+      ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj, true));
+      if (!view.is_utf8) {
+        return internal::InvalidValue(obj, "was not a utf8 string");
+      }
+      return std::move(view);
+    } else {
+      // Non-strict conversion; keep track of whether values are unicode or bytes
+      return PyBytesView::FromString(obj);
     }
-    return Status::OK();
   }
 
-  virtual Status GetResult(std::shared_ptr<ChunkedArray>* out) {
-    // Still some accumulated data in the builder. If there are no chunks, we
-    // always call Finish to deal with the edge case where a size-0 sequence
-    // was converted with a specific output type, like array([], type=t)
-    RETURN_NOT_OK(Close());
-    *out = std::make_shared<ChunkedArray>(this->chunks_, builder_->type());
-    return Status::OK();
+  static Result<bool> Convert(const DataType* type, const O&, I obj) {
+    return Status::NotImplemented("PyValue::Convert is not implemented for type ", type);
   }
-
-  ArrayBuilder* builder() const { return builder_; }
-
-  int num_chunks() const { return static_cast<int>(chunks_.size()); }
-
- protected:
-  ArrayBuilder* builder_;
-  bool unfinished_builder_;
-  std::vector<std::shared_ptr<Array>> chunks_;
 };
 
-template <typename Type, NullCoding null_coding = NullCoding::NONE_ONLY>
-class TypedConverter : public SeqConverter {
- public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  Status Init(ArrayBuilder* builder) override {
-    builder_ = builder;
-    DCHECK_NE(builder_, nullptr);
-    typed_builder_ = checked_cast<BuilderType*>(builder);
-    return Status::OK();
-  }
-
-  // Append a missing item (default implementation)
-  Status AppendNull() override { return this->typed_builder_->AppendNull(); }
-
-  // Append null if the obj is None or pandas null otherwise the valid value
-  Status Append(PyObject* obj) override {
-    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
-  }
-
-  Status Extend(PyObject* obj, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequence(
-        obj, [this](PyObject* item, bool* /* unused */) { return this->Append(item); });
-  }
-
-  Status ExtendMasked(PyObject* obj, PyObject* mask, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequenceMasked(
-        obj, mask, [this](PyObject* item, bool is_masked, bool* /* unused */) {
-          if (is_masked) {
-            return this->AppendNull();
-          } else {
-            // This will also apply the null-checking convention in the event
-            // that the value is not masked
-            return this->Append(item);  // perhaps use AppendValue instead?
-          }
-        });
-  }
+template <typename T>
+Status Extend(T* converter, PyObject* values, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequence(values, [converter](PyObject* item, bool* /* unused */) {
+    return converter->Append(item);
+  });
+}
 
- protected:
-  BuilderType* typed_builder_;
-};
+// Convert and append a sequence of values masked with a numpy array
+template <typename T>
+Status ExtendMasked(T* converter, PyObject* values, PyObject* mask, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequenceMasked(
+      values, mask, [converter](PyObject* item, bool is_masked, bool* /* unused */) {
+        if (is_masked) {
+          return converter->AppendNull();
+        } else {
+          // This will also apply the null-checking convention in the event
+          // that the value is not masked
+          return converter->Append(item);  // perhaps use AppendValue instead?
+        }
+      });
+}
 
-// ----------------------------------------------------------------------
-// Sequence converter for null type
+// The base Converter class is a mixin with predefined behavior and constructors.
+using PyConverter = Converter<PyObject*, PyConversionOptions>;
 
-template <NullCoding null_coding>
-class NullConverter : public TypedConverter<NullType, null_coding> {
- public:
-  Status AppendValue(PyObject* obj) override {
-    return internal::InvalidValue(obj, "converting to null type");
-  }
-};
+template <typename T, typename Enable = void>
+class PyPrimitiveConverter;
 
-// ----------------------------------------------------------------------
-// Sequence converter template for primitive (integer and floating point bool) types
+template <typename T>
+class PyListConverter;
 
-template <typename Type, NullCoding null_coding>
-class PrimitiveConverter : public TypedConverter<Type, null_coding> {
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value, ValueConverter<Type>::FromPython(obj));
-    return this->typed_builder_->Append(value);
-  }
-};
+template <typename U, typename Enable = void>
+class PyDictionaryConverter;
 
-// ----------------------------------------------------------------------
-// Sequence converters for temporal types
+class PyStructConverter;
 
-template <typename Type, NullCoding null_coding>
-class TimeConverter : public TypedConverter<Type, null_coding> {
- public:
-  explicit TimeConverter(TimeUnit::type unit, bool ignore_timezone)
-      : unit_(unit), ignore_timezone_(ignore_timezone) {}
-
-  // TODO(kszucs): support numpy values for date and time converters
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value,
-                          ValueConverter<Type>::FromPython(obj, unit_, ignore_timezone_));
-    return this->typed_builder_->Append(value);
-  }
+template <typename T, typename Enable = void>
+struct PyConverterTrait;
 
- protected:
-  TimeUnit::type unit_;
-  bool ignore_timezone_;
+template <typename T>
+struct PyConverterTrait<
+    T, enable_if_t<!is_nested_type<T>::value && !is_interval_type<T>::value &&
+                   !is_extension_type<T>::value>> {
+  using type = PyPrimitiveConverter<T>;
 };
 
-// TODO(kszucs): move it to the type_traits
 template <typename T>
-struct NumpyType {};
+struct PyConverterTrait<T, enable_if_list_like<T>> {
+  using type = PyListConverter<T>;
+};
 
 template <>
-struct NumpyType<TimestampType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_DATETIME>::isnull(v);
-  }
+struct PyConverterTrait<StructType> {
+  using type = PyStructConverter;
 };
 
 template <>
-struct NumpyType<DurationType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_TIMEDELTA>::isnull(v);
-  }
+struct PyConverterTrait<DictionaryType> {
+  template <typename T>
+  using type = PyDictionaryConverter<T>;
 };
 
-template <typename Type, NullCoding null_coding>
-class TemporalConverter : public TimeConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_null_type<T>::value || is_boolean_type<T>::value ||
+                   is_number_type<T>::value || is_decimal_type<T>::value ||
+                   is_date_type<T>::value || is_time_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using TimeConverter<Type, null_coding>::TimeConverter;
-
-  Status AppendValue(PyObject* obj) override {
-    int64_t value;
-    if (PyArray_CheckAnyScalarExact(obj)) {
-      // convert np.datetime64 / np.timedelta64 depending on Type
-      ARROW_ASSIGN_OR_RAISE(value, ValueConverter<Type>::FromNumpy(obj, this->unit_));
-      if (NumpyType<Type>::isnull(value)) {
-        // checks numpy NaT sentinel after conversion
-        return this->typed_builder_->AppendNull();
-      }
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
     } else {
       ARROW_ASSIGN_OR_RAISE(
-          value,
-          ValueConverter<Type>::FromPython(
-              obj, this->unit_, TimeConverter<Type, null_coding>::ignore_timezone_));
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      return this->primitive_builder_->Append(converted);
     }
-    return this->typed_builder_->Append(value);
   }
 };
 
-// ----------------------------------------------------------------------
-// Sequence converters for Binary, FixedSizeBinary, String
-
-template <typename Type, NullCoding null_coding>
-class BinaryLikeConverter : public TypedConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_timestamp_type<T>::value || is_duration_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  inline Status AutoChunk(Py_ssize_t size) {
-    // did we reach the builder size limit?
-    if (ARROW_PREDICT_FALSE(this->typed_builder_->value_data_length() + size >
-                            BuilderType::memory_limit())) {
-      // builder would be full, so need to add a new chunk
-      std::shared_ptr<Array> chunk;
-      RETURN_NOT_OK(this->typed_builder_->Finish(&chunk));
-      this->chunks_.emplace_back(std::move(chunk));
-    }
-    return Status::OK();
-  }
-
-  Status AppendString(const PyBytesView& view) {
-    // check that the value fits in the datatype
-    if (view.size > BuilderType::memory_limit()) {
-      return Status::Invalid("string too large for datatype");
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      // Numpy NaT sentinels can be checked after the conversion
+      if (PyArray_CheckAnyScalarExact(value) &&
+          PyValue::IsNaT(this->primitive_type_, converted)) {
+        return this->primitive_builder_->AppendNull();
+      } else {
+        return this->primitive_builder_->Append(converted);
+      }
     }
-    DCHECK_GE(view.size, 0);
-
-    // create a new chunk if the value would overflow the builder
-    RETURN_NOT_OK(AutoChunk(view.size));
-
-    // now we can safely append the value to the builder
-    RETURN_NOT_OK(
-        this->typed_builder_->Append(::arrow::util::string_view(view.bytes, view.size)));
-
-    return Status::OK();
   }
-
- protected:
-  // Create a single instance of PyBytesView here to prevent unnecessary object
-  // creation/destruction
-  PyBytesView string_view_;
 };
 
-template <typename Type, NullCoding null_coding>
-class BinaryConverter : public BinaryLikeConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_binary<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto view, ValueConverter<Type>::FromPython(obj));
-    return this->AppendString(view);
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
+    }
   }
 };
 
-template <NullCoding null_coding>
-class FixedSizeBinaryConverter
-    : public BinaryLikeConverter<FixedSizeBinaryType, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_string_like<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  explicit FixedSizeBinaryConverter(int32_t byte_width) : byte_width_(byte_width) {}
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      if (!view.is_utf8) {
+        // observed binary value
+        observed_binary_ = true;
+      }
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
+    }
+  }
 
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(
-        this->string_view_,
-        ValueConverter<FixedSizeBinaryType>::FromPython(obj, byte_width_));
-    return this->AppendString(this->string_view_);
+  Result<std::shared_ptr<Array>> ToArray() override {
+    ARROW_ASSIGN_OR_RAISE(auto array, (PrimitiveConverter<T, PyConverter>::ToArray()));
+    if (observed_binary_) {
+      // if we saw any non-unicode, cast results to BinaryArray
+      auto binary_type = TypeTraits<typename T::PhysicalType>::type_singleton();
+      return array->View(binary_type);
+    } else {
+      return array;
+    }
   }
 
  protected:
-  int32_t byte_width_;
+  bool observed_binary_ = false;
 };
 
-// For String/UTF8, if strict_conversions enabled, we reject any non-UTF8,
-// otherwise we allow but return results as BinaryArray
-template <typename Type, bool Strict, NullCoding null_coding>
-class StringConverter : public BinaryLikeConverter<Type, null_coding> {
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_c_type<U>>
+    : public DictionaryConverter<U, PyConverter> {
  public:
-  StringConverter() : binary_count_(0) {}
-
-  Status AppendValue(PyObject* obj) override {
-    if (Strict) {
-      // raise if the object is not unicode or not an utf-8 encoded bytes
-      ARROW_ASSIGN_OR_RAISE(this->string_view_, ValueConverter<Type>::FromPython(obj));
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
     } else {
-      // keep track of whether values are unicode or bytes; if any bytes are
-      // observe, the result will be bytes
-      bool is_utf8;
-      ARROW_ASSIGN_OR_RAISE(this->string_view_,
-                            ValueConverter<Type>::FromPython(obj, &is_utf8));
-      if (!is_utf8) {
-        ++binary_count_;
-      }
+      ARROW_ASSIGN_OR_RAISE(auto converted,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(converted);
     }
-    return this->AppendString(this->string_view_);
   }
+};
 
-  Status GetResult(std::shared_ptr<ChunkedArray>* out) override {
-    RETURN_NOT_OK(SeqConverter::GetResult(out));
-
-    // If we saw any non-unicode, cast results to BinaryArray
-    if (binary_count_) {
-      // We should have bailed out earlier
-      DCHECK(!Strict);
-      auto binary_type = TypeTraits<typename Type::PhysicalType>::type_singleton();
-      return (*out)->View(binary_type).Value(out);
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_string_view<U>>
+    : public DictionaryConverter<U, PyConverter> {
+ public:
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(auto view,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(util::string_view(view.bytes, view.size));
     }
-    return Status::OK();
   }
-
- protected:
-  int64_t binary_count_;
 };
 
-// ----------------------------------------------------------------------
-// Convert lists (NumPy arrays containing lists or ndarrays as values)
-
 // If the value type does not match the expected NumPy dtype, then fall through
 // to a slower PySequence-based path
-#define LIST_FAST_CASE(TYPE, NUMPY_TYPE, ArrowType)            \
-  case Type::TYPE: {                                           \
-    if (PyArray_DESCR(arr)->type_num != NUMPY_TYPE) {          \
-      return value_converter_->Extend(obj, value_length);      \
-    }                                                          \
-    return AppendNdarrayTypedItem<NUMPY_TYPE, ArrowType>(arr); \
+#define LIST_FAST_CASE(TYPE_ID, TYPE, NUMPY_TYPE)               \
+  case Type::TYPE_ID: {                                         \
+    if (PyArray_DESCR(ndarray)->type_num != NUMPY_TYPE) {       \
+      return Extend(this->value_converter_.get(), value, size); \
+    }                                                           \
+    return AppendNdarrayTyped<TYPE, NUMPY_TYPE>(ndarray);       \
   }
 
 // Use internal::VisitSequence, fast for NPY_OBJECT but slower otherwise
-#define LIST_SLOW_CASE(TYPE)                            \
-  case Type::TYPE: {                                    \
-    return value_converter_->Extend(obj, value_length); \
+#define LIST_SLOW_CASE(TYPE_ID)                               \
+  case Type::TYPE_ID: {                                       \
+    return Extend(this->value_converter_.get(), value, size); \
   }
 
-// Base class for ListConverter and FixedSizeListConverter (to have both work with CRTP)
-template <typename TypeClass, NullCoding null_coding>
-class BaseListConverter : public TypedConverter<TypeClass, null_coding> {
+template <typename T>
+class PyListConverter : public ListConverter<T, PyConverter, PyConverterTrait> {
  public:
-  using BuilderType = typename TypeTraits<TypeClass>::BuilderType;
-
-  explicit BaseListConverter(bool from_pandas, bool strict_conversions,
-                             bool ignore_timezone)
-      : from_pandas_(from_pandas),
-        strict_conversions_(strict_conversions),
-        ignore_timezone_(ignore_timezone) {}
-
-  Status Init(ArrayBuilder* builder) override {
-    this->builder_ = builder;
-    this->typed_builder_ = checked_cast<BuilderType*>(builder);
-
-    this->value_type_ = checked_cast<const TypeClass&>(*builder->type()).value_type();
-    RETURN_NOT_OK(GetConverter(value_type_, from_pandas_, strict_conversions_,
-                               ignore_timezone_, &value_converter_));
-    return this->value_converter_->Init(this->typed_builder_->value_builder());
-  }
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->list_builder_->AppendNull();
+    }
 
-  template <int NUMPY_TYPE, typename Type>
-  Status AppendNdarrayTypedItem(PyArrayObject* arr) {
-    using traits = internal::npy_traits<NUMPY_TYPE>;
-    using T = typename traits::value_type;
-    using ValueBuilderType = typename TypeTraits<Type>::BuilderType;
+    RETURN_NOT_OK(this->list_builder_->Append());
+    if (PyArray_Check(value)) {
+      RETURN_NOT_OK(AppendNdarray(value));
+    } else if (PySequence_Check(value)) {
+      RETURN_NOT_OK(AppendSequence(value));
+    } else {
+      return internal::InvalidType(
+          value, "was not a sequence or recognized null for conversion to list type");
+    }
 
-    const bool null_sentinels_possible =
-        // Always treat Numpy's NaT as null
-        NUMPY_TYPE == NPY_DATETIME || NUMPY_TYPE == NPY_TIMEDELTA ||
-        // Observing pandas's null sentinels
-        (from_pandas_ && traits::supports_nulls);
+    return ValidateBuilder(this->list_type_);
+  }
 
-    auto child_builder = checked_cast<ValueBuilderType*>(value_converter_->builder());
+ protected:
+  Status ValidateOverflow(const MapType*, int64_t size) { return Status::OK(); }

Review comment:
       Also need to write a test case for 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.

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



[GitHub] [arrow] kszucs commented on pull request #8088: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API [WIP]

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


   > Why separate `PyArrayConverter` and `ArrayConverter`?
   
   My hope is that the classes in `converter.h` can be reused in the future to implement different converter APIs, as an example it could be used in the R bindings as well. It would be even better if we could centralize converter agnostic logic here like auto chunking.


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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -329,186 +300,106 @@ struct ValueConverter<DurationType> {
         default:
           return Status::UnknownError("Invalid time unit");
       }
+    } else if (PyArray_CheckAnyScalarExact(obj)) {
+      // validate that the numpy scalar has np.datetime64 dtype
+      std::shared_ptr<DataType> numpy_type;
+      RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &numpy_type));
+      if (!numpy_type->Equals(*type)) {
+        return Status::NotImplemented("Expected np.timedelta64 but got: ",
+                                      numpy_type->ToString());
+      }
+      return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
     } else {
       RETURN_NOT_OK(internal::CIntFromPython(obj, &value));
     }
     return value;
   }
 
-  static inline Result<int64_t> FromNumpy(PyObject* obj, TimeUnit::type unit) {
-    // validate that the numpy scalar has np.timedelta64 dtype
-    std::shared_ptr<DataType> type;
-    RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &type));
-    if (type->id() != DurationType::type_id) {
-      // TODO(kszucs): the message should highlight the received numpy dtype
-      return Status::Invalid("Expected np.timedelta64 but got: ", type->ToString());
-    }
-    // validate that the time units are matching
-    if (unit != checked_cast<const DurationType&>(*type).unit()) {
-      return Status::NotImplemented(
-          "Cannot convert NumPy np.timedelta64 objects with differing unit");
-    }
-    // convert the numpy value
-    return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_any_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    return std::move(view);
-  }
-};
+  // The binary-like intermediate representation is PyBytesView because it keeps temporary
+  // python objects alive (non-contiguous memoryview) and stores whether the original
+  // object was unicode encoded or not, which is used for unicode -> bytes coersion if
+  // there is a non-unicode object observed.
 
-template <typename Type>
-struct ValueConverter<Type, enable_if_string_like<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    // strict conversion, force output to be unicode / utf8 and validate that
-    // any binary values are utf8
-    bool is_utf8 = false;
-    PyBytesView view;
-
-    RETURN_NOT_OK(view.FromString(obj, &is_utf8));
-    if (!is_utf8) {
-      return internal::InvalidValue(obj, "was not a utf8 string");
-    }
-    return std::move(view);
+  static Result<PyBytesView> Convert(const BaseBinaryType*, const O&, I obj) {
+    return PyBytesView::FromString(obj);
   }
 
-  static inline Result<PyBytesView> FromPython(PyObject* obj, bool* is_utf8) {
-    PyBytesView view;
-
-    // Non-strict conversion; keep track of whether values are unicode or bytes
-    if (PyUnicode_Check(obj)) {
-      *is_utf8 = true;
-      RETURN_NOT_OK(view.FromUnicode(obj));
+  static Result<PyBytesView> Convert(const FixedSizeBinaryType* type, const O&, I obj) {
+    ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj));
+    if (ARROW_PREDICT_TRUE(view.size == type->byte_width())) {
+      return std::move(view);
     } else {
-      // If not unicode or bytes, FromBinary will error
-      *is_utf8 = false;
-      RETURN_NOT_OK(view.FromBinary(obj));
-    }
-    return std::move(view);
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_fixed_size_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj, int32_t byte_width) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    if (ARROW_PREDICT_FALSE(view.size != byte_width)) {
       std::stringstream ss;
-      ss << "expected to be length " << byte_width << " was " << view.size;
+      ss << "expected to be length " << type->byte_width() << " was " << view.size;
       return internal::InvalidValue(obj, ss.str());
-    } else {
-      return std::move(view);
     }
   }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter base and CRTP "middle" subclasses
 
-class SeqConverter;
-
-// Forward-declare converter factory
-Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
-                    bool strict_conversions, bool ignore_timezone,
-                    std::unique_ptr<SeqConverter>* out);
-
-// Marshal Python sequence (list, tuple, etc.) to Arrow array
-class SeqConverter {
- public:
-  virtual ~SeqConverter() = default;
-
-  // Initialize the sequence converter with an ArrayBuilder created
-  // externally. The reason for this interface is that we have
-  // arrow::MakeBuilder which also creates child builders for nested types, so
-  // we have to pass in the child builders to child SeqConverter in the case of
-  // converting Python objects to Arrow nested types
-  virtual Status Init(ArrayBuilder* builder) = 0;
-
-  // Append a single null value to the builder
-  virtual Status AppendNull() = 0;
-
-  // Append a valid value
-  virtual Status AppendValue(PyObject* seq) = 0;
-
-  // Append a single python object handling Null values
-  virtual Status Append(PyObject* seq) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status Extend(PyObject* seq, int64_t size) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status ExtendMasked(PyObject* seq, PyObject* mask, int64_t size) = 0;
-
-  virtual Status Close() {
-    if (chunks_.size() == 0 || builder_->length() > 0) {
-      std::shared_ptr<Array> last_chunk;
-      RETURN_NOT_OK(builder_->Finish(&last_chunk));
-      chunks_.emplace_back(std::move(last_chunk));
+  template <typename T>
+  static enable_if_string<T, Result<PyBytesView>> Convert(const T*, const O& options,
+                                                          I obj) {
+    if (options.strict) {
+      // Strict conversion, force output to be unicode / utf8 and validate that
+      // any binary values are utf8
+      ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj, true));
+      if (!view.is_utf8) {
+        return internal::InvalidValue(obj, "was not a utf8 string");
+      }
+      return std::move(view);
+    } else {
+      // Non-strict conversion; keep track of whether values are unicode or bytes
+      return PyBytesView::FromString(obj);
     }
-    return Status::OK();
   }
 
-  virtual Status GetResult(std::shared_ptr<ChunkedArray>* out) {
-    // Still some accumulated data in the builder. If there are no chunks, we
-    // always call Finish to deal with the edge case where a size-0 sequence
-    // was converted with a specific output type, like array([], type=t)
-    RETURN_NOT_OK(Close());
-    *out = std::make_shared<ChunkedArray>(this->chunks_, builder_->type());
-    return Status::OK();
+  static Result<bool> Convert(const DataType* type, const O&, I obj) {
+    return Status::NotImplemented("PyValue::Convert is not implemented for type ", type);
   }
-
-  ArrayBuilder* builder() const { return builder_; }
-
-  int num_chunks() const { return static_cast<int>(chunks_.size()); }
-
- protected:
-  ArrayBuilder* builder_;
-  bool unfinished_builder_;
-  std::vector<std::shared_ptr<Array>> chunks_;
 };
 
-template <typename Type, NullCoding null_coding = NullCoding::NONE_ONLY>
-class TypedConverter : public SeqConverter {
- public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
+// Forward-declare the type-family specific converters to inject them to the PyConverter
+// base class as type aliases.
 
-  Status Init(ArrayBuilder* builder) override {
-    builder_ = builder;
-    DCHECK_NE(builder_, nullptr);
-    typed_builder_ = checked_cast<BuilderType*>(builder);
-    return Status::OK();
-  }
+template <typename T, typename Enable = void>
+class PyPrimitiveConverter;
 
-  // Append a missing item (default implementation)
-  Status AppendNull() override { return this->typed_builder_->AppendNull(); }
+template <typename T, typename Enable = void>
+class PyDictionaryConverter;
 
-  // Append null if the obj is None or pandas null otherwise the valid value
-  Status Append(PyObject* obj) override {
-    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
-  }
+template <typename T>
+class PyListConverter;
 
-  Status Extend(PyObject* obj, int64_t size) override {
+class PyStructConverter;
+
+// The base Converter class is a mixin with predefined behavior and constructors.
+class PyConverter : public Converter<PyObject*, PyConversionOptions, PyConverter> {
+ public:
+  // Type aliases used by the parent Converter mixin's factory.
+  template <typename T>
+  using Primitive = PyPrimitiveConverter<T>;
+  template <typename T>
+  using Dictionary = PyDictionaryConverter<T>;
+  template <typename T>
+  using List = PyListConverter<T>;
+  using Struct = PyStructConverter;
+
+  // Convert and append a sequence of values
+  Status Extend(PyObject* values, int64_t size) {

Review comment:
       Indeed, why I wasn't thinking of this before.




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/array.pxi
##########
@@ -158,24 +158,44 @@ def array(object obj, type=None, mask=None, size=None, from_pandas=None,
     Notes
     -----
     Localized timestamps will currently be returned as UTC (pandas's native
-    representation).  Timezone-naive data will be implicitly interpreted as
+    representation). Timezone-naive data will be implicitly interpreted as
     UTC.
 
+    Converting to dictionary array will promote to a wider integer type for
+    indices if the number of distinct values cannot be represented, even if
+    the index type was explicitly set. This means that if there are more than
+    127 values the returned dictionary array's index type will be at least
+    pa.int16() even if pa.int8() was passed to the function. Note that an
+    explicit index type will not be demoted even if it is wider than required.
+
     Examples
     --------
     >>> import pandas as pd
     >>> import pyarrow as pa
     >>> pa.array(pd.Series([1, 2]))
-    <pyarrow.array.Int64Array object at 0x7f674e4c0e10>
+    <pyarrow.lib.Int64Array object at 0x7f674e4c0e10>
     [
       1,
       2
     ]
 
+    >>> pa.array(["a", "b", "a"], type=pa.dictionary(pa.int8(), pa.string()))
+    <pyarrow.lib.DictionaryArray object at 0x7feb288d9040>
+    -- dictionary:
+    [
+      "a",
+      "b"
+    ]
+    -- indices:
+    [
+      0,
+      1,
+      0
+    ]
+

Review comment:
       Added.




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,
+                            std::shared_ptr<ArrayBuilder> builder,
+                            const std::vector<std::shared_ptr<Self>>& children,
+                            OptionsType options) {
+    type_ = std::move(type);
+    builder_ = std::move(builder);
+    children_ = std::move(children);
+    options_ = std::move(options);
+    return Init();
+  }
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>>& children() const { return children_; }
+
+  Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {

Review comment:
       Updated.

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,
+                            std::shared_ptr<ArrayBuilder> builder,
+                            const std::vector<std::shared_ptr<Self>>& children,
+                            OptionsType options) {
+    type_ = std::move(type);
+    builder_ = std::move(builder);
+    children_ = std::move(children);
+    options_ = std::move(options);
+    return Init();
+  }
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>>& children() const { return children_; }
+
+  Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter, template <typename...> class ConverterTrait>
+struct MakeConverterImpl;
+
+template <typename Converter, template <typename...> class ConverterTrait>
+static Result<std::shared_ptr<Converter>> MakeConverter(
+    std::shared_ptr<DataType> type, MemoryPool* pool,
+    typename Converter::OptionsType options) {
+  std::shared_ptr<Converter> out;
+  MakeConverterImpl<Converter, ConverterTrait> visitor = {type, pool, options, &out};
+  ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+  return out;
+}

Review comment:
       Updated.




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

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



[GitHub] [arrow] bkietz commented on pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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


   https://github.com/apache/arrow/pull/8248


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

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



[GitHub] [arrow] kszucs commented on pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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


   @ursabot build


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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/tests/test_convert_builtin.py
##########
@@ -1513,6 +1519,108 @@ def test_struct_from_tuples():
             pa.array([tup], type=ty)
 
 
+def test_struct_from_list_of_pairs():
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 6), ('b', 'bar'), ('c', False)],
+        None
+    ]
+    arr = pa.array(data, type=ty)
+    assert arr.to_pylist() == [
+        {'a': 5, 'b': 'foo', 'c': True},
+        {'a': 6, 'b': 'bar', 'c': False},
+        None
+    ]
+
+    # test with duplicated field names
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('a', pa.string()),
+        pa.field('b', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('a', 'foo'), ('b', True)],
+        [('a', 6), ('a', 'bar'), ('b', False)],
+    ]
+    arr = pa.array(data, type=ty)
+    with pytest.raises(KeyError):
+        # TODO(kszucs): ARROW-9997
+        arr.to_pylist()
+
+    # test with empty elements
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [],
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 2), ('b', 'baz')],
+        [('a', 1), ('b', 'bar'), ('c', False), ('d', 'julia')],

Review comment:
       It will raise since the key field equality is checked explicitly. It also supports converting less elements than the number of fields.
   
   




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,281 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename Input, typename Options>
+class ArrayConverter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  ArrayConverter(const std::shared_ptr<DataType>& type,
+                 std::shared_ptr<ArrayBuilder> builder, Options options)
+      : sp_type_(type), sp_builder_(builder), options_(options) {}
+
+  virtual ~ArrayConverter() = default;
+  const std::shared_ptr<ArrayBuilder>& builder() const { return sp_builder_; }
+  const std::shared_ptr<DataType>& type() const { return sp_type_; }
+  Options options() const { return options_; }
+
+  virtual Status Init() { return Status::OK(); }
+  virtual Status Reserve(int64_t additional_capacity) = 0;
+  virtual Status Append(InputType value) = 0;
+  virtual Status AppendNull() = 0;
+  virtual Status Extend(Input seq, int64_t size) = 0;
+  virtual Result<std::shared_ptr<Array>> Finish() = 0;
+
+ protected:
+  const std::shared_ptr<DataType> sp_type_;
+  std::shared_ptr<ArrayBuilder> sp_builder_;
+  Options options_;
+};
+
+template <typename T, typename BaseConverter,
+          typename BuilderType = typename TypeTraits<T>::BuilderType>
+class TypedArrayConverter : public BaseConverter {

Review comment:
       ```cpp
   class PyStructArrayConverter : public PyArrayConverter
   ```
   
   Would definitely be nicer and in case of StructType we can indeed omit the template parameter, although we still need a type template parameter for the rest of the converters because of type specific intermediate types (c_type, PyBytesView/string_view) so I'm not sure how could we further simplify the API.
   
   >  I think it'd be better to reuse MakeBuilder for consistency then visit the resulting builder to hydrate the converter hierarchy. For example, we could make sp_builder_ an argument to Init().
   
   That was the original implementation and my first attempt as well, but using the current factory and constructing the converters and builders manually ended up in a cleaner code - I'm open to your suggestion though.
   
   




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter>
+struct MakeConverterImpl;
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  template <typename T>
+  using Primitive = PrimitiveConverter<T, Self>;
+  template <typename T>
+  using List = ListConverter<T, Self>;
+  template <typename T>
+  using Dictionary = DictionaryConverter<T, Self>;
+  using Struct = StructConverter<Self>;
+
+  static Result<std::shared_ptr<Self>> Make(std::shared_ptr<DataType> type,
+                                            MemoryPool* pool, OptionsType options) {
+    std::shared_ptr<Self> out;
+    MakeConverterImpl<Self> visitor = {type, pool, options, &out};
+    ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+    ARROW_RETURN_NOT_OK(out->Init());
+    return out;
+  }
+
+  virtual ~Converter() = default;
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }

Review comment:
       To not require complete type support from the implementor.




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter>
+struct MakeConverterImpl;
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  template <typename T>
+  using Primitive = PrimitiveConverter<T, Self>;
+  template <typename T>
+  using List = ListConverter<T, Self>;
+  template <typename T>
+  using Dictionary = DictionaryConverter<T, Self>;
+  using Struct = StructConverter<Self>;
+
+  static Result<std::shared_ptr<Self>> Make(std::shared_ptr<DataType> type,
+                                            MemoryPool* pool, OptionsType options) {
+    std::shared_ptr<Self> out;
+    MakeConverterImpl<Self> visitor = {type, pool, options, &out};
+    ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+    ARROW_RETURN_NOT_OK(out->Init());
+    return out;
+  }
+
+  virtual ~Converter() = default;
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>> children() const { return children_; }
+
+  virtual Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  virtual Status AppendNull() { return builder_->AppendNull(); }

Review comment:
       Updated.




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/scalar.pxi
##########
@@ -615,7 +614,6 @@ cdef class StructScalar(Scalar, collections.abc.Mapping):
         ----------
         index : Union[int, str]
             Index / position or name of the field.
-

Review comment:
       Updating.




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/scalar.pxi
##########
@@ -610,12 +609,10 @@ cdef class StructScalar(Scalar, collections.abc.Mapping):
     def __getitem__(self, key):
         """
         Return the child value for the given field.
-

Review comment:
       Yes, when I reverted.




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,281 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename Input, typename Options>
+class ArrayConverter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  ArrayConverter(const std::shared_ptr<DataType>& type,
+                 std::shared_ptr<ArrayBuilder> builder, Options options)
+      : sp_type_(type), sp_builder_(builder), options_(options) {}
+
+  virtual ~ArrayConverter() = default;
+  const std::shared_ptr<ArrayBuilder>& builder() const { return sp_builder_; }
+  const std::shared_ptr<DataType>& type() const { return sp_type_; }
+  Options options() const { return options_; }
+
+  virtual Status Init() { return Status::OK(); }
+  virtual Status Reserve(int64_t additional_capacity) = 0;
+  virtual Status Append(InputType value) = 0;
+  virtual Status AppendNull() = 0;
+  virtual Status Extend(Input seq, int64_t size) = 0;
+  virtual Result<std::shared_ptr<Array>> Finish() = 0;
+
+ protected:
+  const std::shared_ptr<DataType> sp_type_;
+  std::shared_ptr<ArrayBuilder> sp_builder_;
+  Options options_;
+};
+
+template <typename T, typename BaseConverter,
+          typename BuilderType = typename TypeTraits<T>::BuilderType>
+class TypedArrayConverter : public BaseConverter {
+ public:
+  TypedArrayConverter(const std::shared_ptr<DataType>& type,
+                      std::shared_ptr<ArrayBuilder> builder,
+                      typename BaseConverter::OptionsType options)
+      : BaseConverter(type, builder, options),
+        type_(checked_cast<const T&>(*type)),
+        builder_(checked_cast<BuilderType*>(builder.get())) {}
+
+  Status Reserve(int64_t additional_capacity) override {
+    return this->builder_->Reserve(additional_capacity);
+  }
+
+  Status AppendNull() override { return this->builder_->AppendNull(); }
+
+  Result<std::shared_ptr<Array>> Finish() override { return builder_->Finish(); };
+
+ protected:
+  const T& type_;
+  BuilderType* builder_;
+};
+
+template <typename T, typename BaseConverter>
+class PrimitiveArrayConverter : public TypedArrayConverter<T, BaseConverter> {
+ public:
+  using TypedArrayConverter<T, BaseConverter>::TypedArrayConverter;
+};
+
+template <typename T, typename BaseConverter>
+class DictionaryArrayConverter
+    : public TypedArrayConverter<DictionaryType, BaseConverter, DictionaryBuilder<T>> {
+ public:
+  DictionaryArrayConverter(const std::shared_ptr<DataType>& type,
+                           std::shared_ptr<ArrayBuilder> builder,
+                           typename BaseConverter::OptionsType options)
+      : TypedArrayConverter<DictionaryType, BaseConverter, DictionaryBuilder<T>>(
+            type, builder, options),
+        value_type_(checked_cast<const T&>(
+            *checked_cast<const DictionaryType&>(*type).value_type())) {}
+
+ protected:
+  const T& value_type_;
+};
+
+template <typename T, typename BaseConverter>
+class ListArrayConverter : public TypedArrayConverter<T, BaseConverter> {
+ public:
+  ListArrayConverter(const std::shared_ptr<DataType>& type,
+                     std::shared_ptr<ArrayBuilder> builder,
+                     std::shared_ptr<BaseConverter> value_converter,
+                     typename BaseConverter::OptionsType options)
+      : TypedArrayConverter<T, BaseConverter>(type, builder, options),
+        value_converter_(std::move(value_converter)) {}
+
+ protected:
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename T, typename BaseConverter>
+class StructArrayConverter : public TypedArrayConverter<T, BaseConverter> {
+ public:
+  StructArrayConverter(const std::shared_ptr<DataType>& type,
+                       std::shared_ptr<ArrayBuilder> builder,
+                       std::vector<std::shared_ptr<BaseConverter>> child_converters,
+                       typename BaseConverter::OptionsType options)
+      : TypedArrayConverter<T, BaseConverter>(type, builder, options),
+        child_converters_(std::move(child_converters)) {}
+
+ protected:
+  std::vector<std::shared_ptr<BaseConverter>> child_converters_;
+};
+
+#define DICTIONARY_CASE(TYPE_ENUM, TYPE_CLASS)                                          \
+  case Type::TYPE_ENUM:                                                                 \
+    out->reset(new DictionaryConverter<TYPE_CLASS>(type, std::move(builder), options)); \
+    break;
+
+template <typename Options, typename BaseConverter,
+          template <typename...> class PrimitiveConverter,
+          template <typename...> class DictionaryConverter,
+          template <typename...> class ListConverter,
+          template <typename...> class StructConverter>

Review comment:
       I'd like to keep the construction code somewhere else than `python_to_arrow.cc`. I moved it into the internal namespace so we can change it anytime. 
   
   I could reduce the four converter kinds to two by specializing TypedConverter for list-like and struct type, but not sure how much would it help. 
   
   The previous code was using static dispatch extensively so I tried to keep it, we could certainly simplify the API by using type visitors but I'm afraid that would cause performance regressions.




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/tests/test_convert_builtin.py
##########
@@ -1513,6 +1519,108 @@ def test_struct_from_tuples():
             pa.array([tup], type=ty)
 
 
+def test_struct_from_list_of_pairs():
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 6), ('b', 'bar'), ('c', False)],
+        None
+    ]
+    arr = pa.array(data, type=ty)
+    assert arr.to_pylist() == [
+        {'a': 5, 'b': 'foo', 'c': True},
+        {'a': 6, 'b': 'bar', 'c': False},
+        None
+    ]
+
+    # test with duplicated field names
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('a', pa.string()),
+        pa.field('b', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('a', 'foo'), ('b', True)],
+        [('a', 6), ('a', 'bar'), ('b', False)],
+    ]
+    arr = pa.array(data, type=ty)
+    with pytest.raises(KeyError):
+        # TODO(kszucs): ARROW-9997
+        arr.to_pylist()
+
+    # test with empty elements
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [],
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 2), ('b', 'baz')],
+        [('a', 1), ('b', 'bar'), ('c', False), ('d', 'julia')],

Review comment:
       > Since the _order_ matters when constructing from pairs (based on the test below), I would personally expect additional elements would also not be allowed (BTW, what actually happens if the additional element is not in the last position? Maybe add a test for that as well?).
   It will raise since the key field equality is checked explicitly. It also supports converting less elements than the number of fields.
   
   

##########
File path: python/pyarrow/tests/test_convert_builtin.py
##########
@@ -1513,6 +1519,108 @@ def test_struct_from_tuples():
             pa.array([tup], type=ty)
 
 
+def test_struct_from_list_of_pairs():
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 6), ('b', 'bar'), ('c', False)],
+        None
+    ]
+    arr = pa.array(data, type=ty)
+    assert arr.to_pylist() == [
+        {'a': 5, 'b': 'foo', 'c': True},
+        {'a': 6, 'b': 'bar', 'c': False},
+        None
+    ]
+
+    # test with duplicated field names
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('a', pa.string()),
+        pa.field('b', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('a', 'foo'), ('b', True)],
+        [('a', 6), ('a', 'bar'), ('b', False)],
+    ]
+    arr = pa.array(data, type=ty)
+    with pytest.raises(KeyError):
+        # TODO(kszucs): ARROW-9997
+        arr.to_pylist()
+
+    # test with empty elements
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [],
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 2), ('b', 'baz')],
+        [('a', 1), ('b', 'bar'), ('c', False), ('d', 'julia')],

Review comment:
       > Since the _order_ matters when constructing from pairs (based on the test below), I would personally expect additional elements would also not be allowed (BTW, what actually happens if the additional element is not in the last position? Maybe add a test for that as well?).
   
   It will raise since the key field equality is checked explicitly. It also supports converting less elements than the number of fields.
   
   




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,
+                            std::shared_ptr<ArrayBuilder> builder,
+                            const std::vector<std::shared_ptr<Self>>& children,
+                            OptionsType options) {
+    type_ = std::move(type);
+    builder_ = std::move(builder);
+    children_ = std::move(children);
+    options_ = std::move(options);
+    return Init();
+  }
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>>& children() const { return children_; }
+
+  Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }

Review comment:
       Updated.




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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/array.pxi
##########
@@ -21,28 +21,28 @@ import warnings
 
 cdef _sequence_to_array(object sequence, object mask, object size,
                         DataType type, CMemoryPool* pool, c_bool from_pandas):
-    cdef int64_t c_size
-    cdef PyConversionOptions options
+    cdef:
+        int64_t c_size
+        PyConversionOptions options
+        shared_ptr[CChunkedArray] chunked
 
     if type is not None:
         options.type = type.sp_type
 
     if size is not None:
         options.size = size
 
-    options.pool = pool
     options.from_pandas = from_pandas
-    options.ignore_timezone = os.environ.get('PYARROW_IGNORE_TIMEZONE', False)

Review comment:
       Thanks for the test! Looks good to me

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -615,7 +614,6 @@ cdef class StructScalar(Scalar, collections.abc.Mapping):
         ----------
         index : Union[int, str]
             Index / position or name of the field.
-

Review comment:
       another whitespace revert left over




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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/tests/test_types.py
##########
@@ -280,6 +284,13 @@ def test_tzinfo_to_string_errors():
             pa.lib.tzinfo_to_string(tz)
 
 
+@h.given(tzst.timezones())
+def test_timezone_roundtrip(tz):

Review comment:
       this is specifically `pytz` timezone roundtrip, can you clarify that in the test name or a comment?

##########
File path: python/pyarrow/array.pxi
##########
@@ -21,28 +21,28 @@ import warnings
 
 cdef _sequence_to_array(object sequence, object mask, object size,
                         DataType type, CMemoryPool* pool, c_bool from_pandas):
-    cdef int64_t c_size
-    cdef PyConversionOptions options
+    cdef:
+        int64_t c_size
+        PyConversionOptions options
+        shared_ptr[CChunkedArray] chunked
 
     if type is not None:
         options.type = type.sp_type
 
     if size is not None:
         options.size = size
 
-    options.pool = pool
     options.from_pandas = from_pandas
-    options.ignore_timezone = os.environ.get('PYARROW_IGNORE_TIMEZONE', False)

Review comment:
       What's the reason this is being removed? It's handled elsewhere now?

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -610,12 +609,10 @@ cdef class StructScalar(Scalar, collections.abc.Mapping):
     def __getitem__(self, key):
         """
         Return the child value for the given field.
-

Review comment:
       this whitespace was removed by accident? (in any case rst requires an empty line here) Same fore a few lines below

##########
File path: python/pyarrow/array.pxi
##########
@@ -158,24 +158,44 @@ def array(object obj, type=None, mask=None, size=None, from_pandas=None,
     Notes
     -----
     Localized timestamps will currently be returned as UTC (pandas's native
-    representation).  Timezone-naive data will be implicitly interpreted as
+    representation). Timezone-naive data will be implicitly interpreted as
     UTC.
 
+    Converting to dictionary array will promote to a wider integer type for
+    indices if the number of distinct values cannot be represented, even if
+    the index type was explicitly set. This means that if there are more than
+    127 values the returned dictionary array's index type will be at least
+    pa.int16() even if pa.int8() was passed to the function. Note that an
+    explicit index type will not be demoted even if it is wider than required.
+
     Examples
     --------
     >>> import pandas as pd
     >>> import pyarrow as pa
     >>> pa.array(pd.Series([1, 2]))
-    <pyarrow.array.Int64Array object at 0x7f674e4c0e10>
+    <pyarrow.lib.Int64Array object at 0x7f674e4c0e10>
     [
       1,
       2
     ]
 
+    >>> pa.array(["a", "b", "a"], type=pa.dictionary(pa.int8(), pa.string()))
+    <pyarrow.lib.DictionaryArray object at 0x7feb288d9040>
+    -- dictionary:
+    [
+      "a",
+      "b"
+    ]
+    -- indices:
+    [
+      0,
+      1,
+      0
+    ]
+
     >>> import numpy as np
-    >>> pa.array(pd.Series([1, 2]), np.array([0, 1],
-    ... dtype=bool))
-    <pyarrow.array.Int64Array object at 0x7f9019e11208>
+    >>> pa.array(pd.Series([1, 2]), np.array([0, 1], dtype=bool))

Review comment:
       ```suggestion
       >>> pa.array(pd.Series([1, 2]), mask=np.array([0, 1], dtype=bool))
   ```
   
   this example actually doesn't work otherwise

##########
File path: python/pyarrow/tests/test_convert_builtin.py
##########
@@ -1513,6 +1519,108 @@ def test_struct_from_tuples():
             pa.array([tup], type=ty)
 
 
+def test_struct_from_list_of_pairs():
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 6), ('b', 'bar'), ('c', False)],
+        None
+    ]
+    arr = pa.array(data, type=ty)
+    assert arr.to_pylist() == [
+        {'a': 5, 'b': 'foo', 'c': True},
+        {'a': 6, 'b': 'bar', 'c': False},
+        None
+    ]
+
+    # test with duplicated field names
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('a', pa.string()),
+        pa.field('b', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('a', 'foo'), ('b', True)],
+        [('a', 6), ('a', 'bar'), ('b', False)],
+    ]
+    arr = pa.array(data, type=ty)
+    with pytest.raises(KeyError):
+        # TODO(kszucs): ARROW-9997
+        arr.to_pylist()
+
+    # test with empty elements
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [],
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 2), ('b', 'baz')],
+        [('a', 1), ('b', 'bar'), ('c', False), ('d', 'julia')],

Review comment:
       It doesn't raise on "d" being present here?




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

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



[GitHub] [arrow] bkietz closed pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

Posted by GitBox <gi...@apache.org>.
bkietz closed pull request #8088:
URL: https://github.com/apache/arrow/pull/8088


   


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

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



[GitHub] [arrow] bkietz commented on a change in pull request #8088: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,281 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename Input, typename Options>
+class ArrayConverter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  ArrayConverter(const std::shared_ptr<DataType>& type,
+                 std::shared_ptr<ArrayBuilder> builder, Options options)
+      : sp_type_(type), sp_builder_(builder), options_(options) {}
+
+  virtual ~ArrayConverter() = default;
+  const std::shared_ptr<ArrayBuilder>& builder() const { return sp_builder_; }
+  const std::shared_ptr<DataType>& type() const { return sp_type_; }
+  Options options() const { return options_; }
+
+  virtual Status Init() { return Status::OK(); }
+  virtual Status Reserve(int64_t additional_capacity) = 0;
+  virtual Status Append(InputType value) = 0;
+  virtual Status AppendNull() = 0;
+  virtual Status Extend(Input seq, int64_t size) = 0;
+  virtual Result<std::shared_ptr<Array>> Finish() = 0;
+
+ protected:
+  const std::shared_ptr<DataType> sp_type_;
+  std::shared_ptr<ArrayBuilder> sp_builder_;
+  Options options_;
+};
+
+template <typename T, typename BaseConverter,
+          typename BuilderType = typename TypeTraits<T>::BuilderType>
+class TypedArrayConverter : public BaseConverter {
+ public:
+  TypedArrayConverter(const std::shared_ptr<DataType>& type,
+                      std::shared_ptr<ArrayBuilder> builder,
+                      typename BaseConverter::OptionsType options)
+      : BaseConverter(type, builder, options),
+        type_(checked_cast<const T&>(*type)),
+        builder_(checked_cast<BuilderType*>(builder.get())) {}
+
+  Status Reserve(int64_t additional_capacity) override {
+    return this->builder_->Reserve(additional_capacity);
+  }
+
+  Status AppendNull() override { return this->builder_->AppendNull(); }
+
+  Result<std::shared_ptr<Array>> Finish() override { return builder_->Finish(); };
+
+ protected:
+  const T& type_;
+  BuilderType* builder_;
+};
+
+template <typename T, typename BaseConverter>
+class PrimitiveArrayConverter : public TypedArrayConverter<T, BaseConverter> {
+ public:
+  using TypedArrayConverter<T, BaseConverter>::TypedArrayConverter;
+};
+
+template <typename T, typename BaseConverter>
+class DictionaryArrayConverter
+    : public TypedArrayConverter<DictionaryType, BaseConverter, DictionaryBuilder<T>> {
+ public:
+  DictionaryArrayConverter(const std::shared_ptr<DataType>& type,
+                           std::shared_ptr<ArrayBuilder> builder,
+                           typename BaseConverter::OptionsType options)
+      : TypedArrayConverter<DictionaryType, BaseConverter, DictionaryBuilder<T>>(
+            type, builder, options),
+        value_type_(checked_cast<const T&>(
+            *checked_cast<const DictionaryType&>(*type).value_type())) {}
+
+ protected:
+  const T& value_type_;
+};
+
+template <typename T, typename BaseConverter>
+class ListArrayConverter : public TypedArrayConverter<T, BaseConverter> {
+ public:
+  ListArrayConverter(const std::shared_ptr<DataType>& type,
+                     std::shared_ptr<ArrayBuilder> builder,
+                     std::shared_ptr<BaseConverter> value_converter,
+                     typename BaseConverter::OptionsType options)
+      : TypedArrayConverter<T, BaseConverter>(type, builder, options),
+        value_converter_(std::move(value_converter)) {}
+
+ protected:
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename T, typename BaseConverter>
+class StructArrayConverter : public TypedArrayConverter<T, BaseConverter> {
+ public:
+  StructArrayConverter(const std::shared_ptr<DataType>& type,
+                       std::shared_ptr<ArrayBuilder> builder,
+                       std::vector<std::shared_ptr<BaseConverter>> child_converters,
+                       typename BaseConverter::OptionsType options)
+      : TypedArrayConverter<T, BaseConverter>(type, builder, options),
+        child_converters_(std::move(child_converters)) {}
+
+ protected:
+  std::vector<std::shared_ptr<BaseConverter>> child_converters_;
+};
+
+#define DICTIONARY_CASE(TYPE_ENUM, TYPE_CLASS)                                          \
+  case Type::TYPE_ENUM:                                                                 \
+    out->reset(new DictionaryConverter<TYPE_CLASS>(type, std::move(builder), options)); \
+    break;
+
+template <typename Options, typename BaseConverter,
+          template <typename...> class PrimitiveConverter,
+          template <typename...> class DictionaryConverter,
+          template <typename...> class ListConverter,
+          template <typename...> class StructConverter>

Review comment:
       Instead of requiring concrete converters be class templates like this, please let generic construction of converters wait for a follow up. Construct ArrayBuilders using `MakeBuilder` and move any other converter construction code from ArrayConverterBuilder into `python_to_arrow.cc`.

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,281 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename Input, typename Options>
+class ArrayConverter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  ArrayConverter(const std::shared_ptr<DataType>& type,
+                 std::shared_ptr<ArrayBuilder> builder, Options options)
+      : sp_type_(type), sp_builder_(builder), options_(options) {}
+
+  virtual ~ArrayConverter() = default;
+  const std::shared_ptr<ArrayBuilder>& builder() const { return sp_builder_; }
+  const std::shared_ptr<DataType>& type() const { return sp_type_; }
+  Options options() const { return options_; }
+
+  virtual Status Init() { return Status::OK(); }
+  virtual Status Reserve(int64_t additional_capacity) = 0;
+  virtual Status Append(InputType value) = 0;
+  virtual Status AppendNull() = 0;
+  virtual Status Extend(Input seq, int64_t size) = 0;

Review comment:
       I don't think it makes sense to make `Extend()` a method of this interface: I wouldn't expect that all converted `InputType`s are unboxable as sequences or scalars in the way that `PyObject*` happens to be. Even in the specific case of `PyObject*` which has this property, Extend() needs to be supplemented with ExtendMasked() if a mask is present in addition to the sequence. I think Extend() should be moved to PyArrayConverter for now

##########
File path: cpp/src/arrow/util/hashing.h
##########
@@ -851,6 +851,11 @@ struct HashTraits<T, enable_if_t<has_string_view<T>::value &&
   using MemoTableType = BinaryMemoTable<BinaryBuilder>;
 };
 
+template <>
+struct HashTraits<Decimal128Type> {
+  using MemoTableType = BinaryMemoTable<BinaryBuilder>;

Review comment:
       `Decimal128` is pretty small, I think this would work:
   ```suggestion
     using MemoTableType = ScalarMemoTable<Decimal128, HashTable>;
   ```

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,281 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename Input, typename Options>
+class ArrayConverter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  ArrayConverter(const std::shared_ptr<DataType>& type,
+                 std::shared_ptr<ArrayBuilder> builder, Options options)
+      : sp_type_(type), sp_builder_(builder), options_(options) {}
+
+  virtual ~ArrayConverter() = default;
+  const std::shared_ptr<ArrayBuilder>& builder() const { return sp_builder_; }
+  const std::shared_ptr<DataType>& type() const { return sp_type_; }
+  Options options() const { return options_; }
+
+  virtual Status Init() { return Status::OK(); }
+  virtual Status Reserve(int64_t additional_capacity) = 0;
+  virtual Status Append(InputType value) = 0;
+  virtual Status AppendNull() = 0;
+  virtual Status Extend(Input seq, int64_t size) = 0;
+  virtual Result<std::shared_ptr<Array>> Finish() = 0;
+
+ protected:
+  const std::shared_ptr<DataType> sp_type_;
+  std::shared_ptr<ArrayBuilder> sp_builder_;
+  Options options_;
+};
+
+template <typename T, typename BaseConverter,
+          typename BuilderType = typename TypeTraits<T>::BuilderType>
+class TypedArrayConverter : public BaseConverter {

Review comment:
       I don't think the TypedArrayConverter and its subclasses need to be public. This framework would be more understandable if (for example) instead of
   ```c++
   template <typename T>
   class PyStructArrayConverter : public StructArrayConverter<T, PyArrayConverter> {
   ```
   we had
   ```c++
   class PyStructArrayConverter : public PyArrayConverter
   ```
   
   It seems the `TypedArrayConverter`s and base classes are mainly defined to enforce regular construction of the various converter types and to allow child builders to be wrapped with the corresponding child converters. Instead of introducing ArrayConverterBuilder, I think it'd be better to reuse `MakeBuilder` for consistency then visit the resulting builder to hydrate the converter hierarchy. For example, we could make `sp_builder_` an argument to `Init()`.
   
   In any case, IMHO the number and complexity of base classes should be kept to a minimum to simplify the implementation process; for example it's surprising that PyStructArrayConverter must be a template which is only instantiated with `argument=StructType`

##########
File path: cpp/src/arrow/python/python_test.cc
##########
@@ -422,17 +421,15 @@ TEST_F(DecimalTest, TestNoneAndNaN) {
   ASSERT_EQ(0, PyList_SetItem(list, 2, missing_value2));
   ASSERT_EQ(0, PyList_SetItem(list, 3, missing_value3));
 
-  std::shared_ptr<ChunkedArray> arr, arr_from_pandas;
   PyConversionOptions options;
-  ASSERT_RAISES(TypeError, ConvertPySequence(list, options, &arr));
+  ASSERT_RAISES(TypeError, ConvertPySequence(list, nullptr, options));
 
   options.from_pandas = true;
-  ASSERT_OK(ConvertPySequence(list, options, &arr_from_pandas));
-  auto c0 = arr_from_pandas->chunk(0);
-  ASSERT_TRUE(c0->IsValid(0));
-  ASSERT_TRUE(c0->IsNull(1));
-  ASSERT_TRUE(c0->IsNull(2));
-  ASSERT_TRUE(c0->IsNull(3));
+  auto arr = ConvertPySequence(list, nullptr, options).ValueOrDie();

Review comment:
       ```suggestion
     ASSERT_OK_AND_ASSIGN(auto arr, ConvertPySequence(list, nullptr, options));
   ```

##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -1347,64 +841,49 @@ Status ConvertToSequenceAndInferSize(PyObject* obj, PyObject** seq, int64_t* siz
   return Status::OK();
 }
 
-Status ConvertPySequence(PyObject* sequence_source, PyObject* mask,
-                         const PyConversionOptions& options,
-                         std::shared_ptr<ChunkedArray>* out) {
+Result<std::shared_ptr<Array>> ConvertPySequence(PyObject* obj, PyObject* mask,
+                                                 const PyConversionOptions& opts) {
   PyAcquireGIL lock;
 
   PyObject* seq;
   OwnedRef tmp_seq_nanny;
+  PyConversionOptions options = opts;  // copy options struct since we modify it below
 
   std::shared_ptr<DataType> real_type;
 
   int64_t size = options.size;
-  RETURN_NOT_OK(ConvertToSequenceAndInferSize(sequence_source, &seq, &size));
+  RETURN_NOT_OK(ConvertToSequenceAndInferSize(obj, &seq, &size));
   tmp_seq_nanny.reset(seq);
 
   // In some cases, type inference may be "loose", like strings. If the user
   // passed pa.string(), then we will error if we encounter any non-UTF8
   // value. If not, then we will allow the result to be a BinaryArray
-  bool strict_conversions = false;
+  auto copied_options = options;
+  options.strict = false;
 
   if (options.type == nullptr) {
     RETURN_NOT_OK(InferArrowType(seq, mask, options.from_pandas, &real_type));
-    if (options.ignore_timezone && real_type->id() == Type::TIMESTAMP) {
-      const auto& ts_type = checked_cast<const TimestampType&>(*real_type);
-      real_type = timestamp(ts_type.unit());
-    }
+    // TODO(kszucs): remove this
+    // if (options.ignore_timezone && real_type->id() == Type::TIMESTAMP) {
+    //   const auto& ts_type = checked_cast<const TimestampType&>(*real_type);
+    //   real_type = timestamp(ts_type.unit());
+    // }

Review comment:
       Is this tested?




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter>
+struct MakeConverterImpl;
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  template <typename T>
+  using Primitive = PrimitiveConverter<T, Self>;
+  template <typename T>
+  using List = ListConverter<T, Self>;
+  template <typename T>
+  using Dictionary = DictionaryConverter<T, Self>;
+  using Struct = StructConverter<Self>;
+
+  static Result<std::shared_ptr<Self>> Make(std::shared_ptr<DataType> type,
+                                            MemoryPool* pool, OptionsType options) {
+    std::shared_ptr<Self> out;
+    MakeConverterImpl<Self> visitor = {type, pool, options, &out};
+    ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+    ARROW_RETURN_NOT_OK(out->Init());
+    return out;
+  }
+
+  virtual ~Converter() = default;
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }

Review comment:
       Ok, updating. 

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -610,12 +609,10 @@ cdef class StructScalar(Scalar, collections.abc.Mapping):
     def __getitem__(self, key):
         """
         Return the child value for the given field.
-

Review comment:
       Yes, when I reverted.

##########
File path: python/pyarrow/array.pxi
##########
@@ -21,28 +21,28 @@ import warnings
 
 cdef _sequence_to_array(object sequence, object mask, object size,
                         DataType type, CMemoryPool* pool, c_bool from_pandas):
-    cdef int64_t c_size
-    cdef PyConversionOptions options
+    cdef:
+        int64_t c_size
+        PyConversionOptions options
+        shared_ptr[CChunkedArray] chunked
 
     if type is not None:
         options.type = type.sp_type
 
     if size is not None:
         options.size = size
 
-    options.pool = pool
     options.from_pandas = from_pandas
-    options.ignore_timezone = os.environ.get('PYARROW_IGNORE_TIMEZONE', False)

Review comment:
       We still need it for 2.0.0, so reverting. Nice catch :)

##########
File path: python/pyarrow/tests/test_convert_builtin.py
##########
@@ -1513,6 +1519,108 @@ def test_struct_from_tuples():
             pa.array([tup], type=ty)
 
 
+def test_struct_from_list_of_pairs():
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 6), ('b', 'bar'), ('c', False)],
+        None
+    ]
+    arr = pa.array(data, type=ty)
+    assert arr.to_pylist() == [
+        {'a': 5, 'b': 'foo', 'c': True},
+        {'a': 6, 'b': 'bar', 'c': False},
+        None
+    ]
+
+    # test with duplicated field names
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('a', pa.string()),
+        pa.field('b', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('a', 'foo'), ('b', True)],
+        [('a', 6), ('a', 'bar'), ('b', False)],
+    ]
+    arr = pa.array(data, type=ty)
+    with pytest.raises(KeyError):
+        # TODO(kszucs): ARROW-9997
+        arr.to_pylist()
+
+    # test with empty elements
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [],
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 2), ('b', 'baz')],
+        [('a', 1), ('b', 'bar'), ('c', False), ('d', 'julia')],

Review comment:
       It tolerates extra items (similarly when converting from dicts). I can restrict it if you think that would be more appropiate. We may want to alter this behavior with specific conversion flags.

##########
File path: python/pyarrow/array.pxi
##########
@@ -158,24 +158,44 @@ def array(object obj, type=None, mask=None, size=None, from_pandas=None,
     Notes
     -----
     Localized timestamps will currently be returned as UTC (pandas's native
-    representation).  Timezone-naive data will be implicitly interpreted as
+    representation). Timezone-naive data will be implicitly interpreted as
     UTC.
 
+    Converting to dictionary array will promote to a wider integer type for
+    indices if the number of distinct values cannot be represented, even if
+    the index type was explicitly set. This means that if there are more than
+    127 values the returned dictionary array's index type will be at least
+    pa.int16() even if pa.int8() was passed to the function. Note that an
+    explicit index type will not be demoted even if it is wider than required.
+
     Examples
     --------
     >>> import pandas as pd
     >>> import pyarrow as pa
     >>> pa.array(pd.Series([1, 2]))
-    <pyarrow.array.Int64Array object at 0x7f674e4c0e10>
+    <pyarrow.lib.Int64Array object at 0x7f674e4c0e10>
     [
       1,
       2
     ]
 
+    >>> pa.array(["a", "b", "a"], type=pa.dictionary(pa.int8(), pa.string()))
+    <pyarrow.lib.DictionaryArray object at 0x7feb288d9040>
+    -- dictionary:
+    [
+      "a",
+      "b"
+    ]
+    -- indices:
+    [
+      0,
+      1,
+      0
+    ]
+
     >>> import numpy as np
-    >>> pa.array(pd.Series([1, 2]), np.array([0, 1],
-    ... dtype=bool))
-    <pyarrow.array.Int64Array object at 0x7f9019e11208>
+    >>> pa.array(pd.Series([1, 2]), np.array([0, 1], dtype=bool))

Review comment:
       Updated.

##########
File path: python/pyarrow/tests/test_types.py
##########
@@ -280,6 +284,13 @@ def test_tzinfo_to_string_errors():
             pa.lib.tzinfo_to_string(tz)
 
 
+@h.given(tzst.timezones())
+def test_timezone_roundtrip(tz):

Review comment:
       Updated.

##########
File path: python/pyarrow/array.pxi
##########
@@ -21,28 +21,28 @@ import warnings
 
 cdef _sequence_to_array(object sequence, object mask, object size,
                         DataType type, CMemoryPool* pool, c_bool from_pandas):
-    cdef int64_t c_size
-    cdef PyConversionOptions options
+    cdef:
+        int64_t c_size
+        PyConversionOptions options
+        shared_ptr[CChunkedArray] chunked
 
     if type is not None:
         options.type = type.sp_type
 
     if size is not None:
         options.size = size
 
-    options.pool = pool
     options.from_pandas = from_pandas
-    options.ignore_timezone = os.environ.get('PYARROW_IGNORE_TIMEZONE', False)

Review comment:
       AFAIK It is only tested from the spark integration tests. I can add a unittest for it. 

##########
File path: python/pyarrow/tests/test_convert_builtin.py
##########
@@ -1513,6 +1519,108 @@ def test_struct_from_tuples():
             pa.array([tup], type=ty)
 
 
+def test_struct_from_list_of_pairs():
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 6), ('b', 'bar'), ('c', False)],
+        None
+    ]
+    arr = pa.array(data, type=ty)
+    assert arr.to_pylist() == [
+        {'a': 5, 'b': 'foo', 'c': True},
+        {'a': 6, 'b': 'bar', 'c': False},
+        None
+    ]
+
+    # test with duplicated field names
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('a', pa.string()),
+        pa.field('b', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('a', 'foo'), ('b', True)],
+        [('a', 6), ('a', 'bar'), ('b', False)],
+    ]
+    arr = pa.array(data, type=ty)
+    with pytest.raises(KeyError):
+        # TODO(kszucs): ARROW-9997
+        arr.to_pylist()
+
+    # test with empty elements
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [],
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 2), ('b', 'baz')],
+        [('a', 1), ('b', 'bar'), ('c', False), ('d', 'julia')],

Review comment:
       > Since the _order_ matters when constructing from pairs (based on the test below), I would personally expect additional elements would also not be allowed (BTW, what actually happens if the additional element is not in the last position? Maybe add a test for that as well?).
   It will raise since the key field equality is checked explicitly. It also supports converting less elements than the number of fields.
   
   

##########
File path: python/pyarrow/tests/test_convert_builtin.py
##########
@@ -1513,6 +1519,108 @@ def test_struct_from_tuples():
             pa.array([tup], type=ty)
 
 
+def test_struct_from_list_of_pairs():
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 6), ('b', 'bar'), ('c', False)],
+        None
+    ]
+    arr = pa.array(data, type=ty)
+    assert arr.to_pylist() == [
+        {'a': 5, 'b': 'foo', 'c': True},
+        {'a': 6, 'b': 'bar', 'c': False},
+        None
+    ]
+
+    # test with duplicated field names
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('a', pa.string()),
+        pa.field('b', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('a', 'foo'), ('b', True)],
+        [('a', 6), ('a', 'bar'), ('b', False)],
+    ]
+    arr = pa.array(data, type=ty)
+    with pytest.raises(KeyError):
+        # TODO(kszucs): ARROW-9997
+        arr.to_pylist()
+
+    # test with empty elements
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [],
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 2), ('b', 'baz')],
+        [('a', 1), ('b', 'bar'), ('c', False), ('d', 'julia')],

Review comment:
       > Since the _order_ matters when constructing from pairs (based on the test below), I would personally expect additional elements would also not be allowed (BTW, what actually happens if the additional element is not in the last position? Maybe add a test for that as well?).
   
   It will raise since the key field equality is checked explicitly. It also supports converting less elements than the number of fields.
   
   

##########
File path: python/pyarrow/tests/test_convert_builtin.py
##########
@@ -1513,6 +1519,108 @@ def test_struct_from_tuples():
             pa.array([tup], type=ty)
 
 
+def test_struct_from_list_of_pairs():
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 6), ('b', 'bar'), ('c', False)],
+        None
+    ]
+    arr = pa.array(data, type=ty)
+    assert arr.to_pylist() == [
+        {'a': 5, 'b': 'foo', 'c': True},
+        {'a': 6, 'b': 'bar', 'c': False},
+        None
+    ]
+
+    # test with duplicated field names
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('a', pa.string()),
+        pa.field('b', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('a', 'foo'), ('b', True)],
+        [('a', 6), ('a', 'bar'), ('b', False)],
+    ]
+    arr = pa.array(data, type=ty)
+    with pytest.raises(KeyError):
+        # TODO(kszucs): ARROW-9997
+        arr.to_pylist()
+
+    # test with empty elements
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [],
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 2), ('b', 'baz')],
+        [('a', 1), ('b', 'bar'), ('c', False), ('d', 'julia')],

Review comment:
       It will raise since the key field equality is checked explicitly. It also supports converting less elements than the number of fields.
   
   

##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -329,985 +302,602 @@ struct ValueConverter<DurationType> {
         default:
           return Status::UnknownError("Invalid time unit");
       }
+    } else if (PyArray_CheckAnyScalarExact(obj)) {
+      // validate that the numpy scalar has np.datetime64 dtype
+      std::shared_ptr<DataType> numpy_type;
+      RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &numpy_type));
+      if (!numpy_type->Equals(*type)) {
+        return Status::NotImplemented("Expected np.timedelta64 but got: ",
+                                      numpy_type->ToString());
+      }
+      return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
     } else {
       RETURN_NOT_OK(internal::CIntFromPython(obj, &value));
     }
     return value;
   }
 
-  static inline Result<int64_t> FromNumpy(PyObject* obj, TimeUnit::type unit) {
-    // validate that the numpy scalar has np.timedelta64 dtype
-    std::shared_ptr<DataType> type;
-    RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &type));
-    if (type->id() != DurationType::type_id) {
-      // TODO(kszucs): the message should highlight the received numpy dtype
-      return Status::Invalid("Expected np.timedelta64 but got: ", type->ToString());
-    }
-    // validate that the time units are matching
-    if (unit != checked_cast<const DurationType&>(*type).unit()) {
-      return Status::NotImplemented(
-          "Cannot convert NumPy np.timedelta64 objects with differing unit");
-    }
-    // convert the numpy value
-    return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_any_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    return std::move(view);
-  }
-};
+  // The binary-like intermediate representation is PyBytesView because it keeps temporary
+  // python objects alive (non-contiguous memoryview) and stores whether the original
+  // object was unicode encoded or not, which is used for unicode -> bytes coersion if
+  // there is a non-unicode object observed.
 
-template <typename Type>
-struct ValueConverter<Type, enable_if_string_like<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    // strict conversion, force output to be unicode / utf8 and validate that
-    // any binary values are utf8
-    bool is_utf8 = false;
-    PyBytesView view;
-
-    RETURN_NOT_OK(view.FromString(obj, &is_utf8));
-    if (!is_utf8) {
-      return internal::InvalidValue(obj, "was not a utf8 string");
-    }
-    return std::move(view);
+  static Result<PyBytesView> Convert(const BaseBinaryType*, const O&, I obj) {
+    return PyBytesView::FromString(obj);
   }
 
-  static inline Result<PyBytesView> FromPython(PyObject* obj, bool* is_utf8) {
-    PyBytesView view;
-
-    // Non-strict conversion; keep track of whether values are unicode or bytes
-    if (PyUnicode_Check(obj)) {
-      *is_utf8 = true;
-      RETURN_NOT_OK(view.FromUnicode(obj));
+  static Result<PyBytesView> Convert(const FixedSizeBinaryType* type, const O&, I obj) {
+    ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj));
+    if (ARROW_PREDICT_TRUE(view.size == type->byte_width())) {
+      return std::move(view);
     } else {
-      // If not unicode or bytes, FromBinary will error
-      *is_utf8 = false;
-      RETURN_NOT_OK(view.FromBinary(obj));
-    }
-    return std::move(view);
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_fixed_size_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj, int32_t byte_width) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    if (ARROW_PREDICT_FALSE(view.size != byte_width)) {
       std::stringstream ss;
-      ss << "expected to be length " << byte_width << " was " << view.size;
+      ss << "expected to be length " << type->byte_width() << " was " << view.size;
       return internal::InvalidValue(obj, ss.str());
-    } else {
-      return std::move(view);
     }
   }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter base and CRTP "middle" subclasses
-
-class SeqConverter;
-
-// Forward-declare converter factory
-Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
-                    bool strict_conversions, bool ignore_timezone,
-                    std::unique_ptr<SeqConverter>* out);
-
-// Marshal Python sequence (list, tuple, etc.) to Arrow array
-class SeqConverter {
- public:
-  virtual ~SeqConverter() = default;
-
-  // Initialize the sequence converter with an ArrayBuilder created
-  // externally. The reason for this interface is that we have
-  // arrow::MakeBuilder which also creates child builders for nested types, so
-  // we have to pass in the child builders to child SeqConverter in the case of
-  // converting Python objects to Arrow nested types
-  virtual Status Init(ArrayBuilder* builder) = 0;
-
-  // Append a single null value to the builder
-  virtual Status AppendNull() = 0;
-
-  // Append a valid value
-  virtual Status AppendValue(PyObject* seq) = 0;
-
-  // Append a single python object handling Null values
-  virtual Status Append(PyObject* seq) = 0;
 
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status Extend(PyObject* seq, int64_t size) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status ExtendMasked(PyObject* seq, PyObject* mask, int64_t size) = 0;
-
-  virtual Status Close() {
-    if (chunks_.size() == 0 || builder_->length() > 0) {
-      std::shared_ptr<Array> last_chunk;
-      RETURN_NOT_OK(builder_->Finish(&last_chunk));
-      chunks_.emplace_back(std::move(last_chunk));
+  template <typename T>
+  static enable_if_string<T, Result<PyBytesView>> Convert(const T*, const O& options,
+                                                          I obj) {
+    if (options.strict) {
+      // Strict conversion, force output to be unicode / utf8 and validate that
+      // any binary values are utf8
+      ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj, true));
+      if (!view.is_utf8) {
+        return internal::InvalidValue(obj, "was not a utf8 string");
+      }
+      return std::move(view);
+    } else {
+      // Non-strict conversion; keep track of whether values are unicode or bytes
+      return PyBytesView::FromString(obj);
     }
-    return Status::OK();
   }
 
-  virtual Status GetResult(std::shared_ptr<ChunkedArray>* out) {
-    // Still some accumulated data in the builder. If there are no chunks, we
-    // always call Finish to deal with the edge case where a size-0 sequence
-    // was converted with a specific output type, like array([], type=t)
-    RETURN_NOT_OK(Close());
-    *out = std::make_shared<ChunkedArray>(this->chunks_, builder_->type());
-    return Status::OK();
+  static Result<bool> Convert(const DataType* type, const O&, I obj) {
+    return Status::NotImplemented("PyValue::Convert is not implemented for type ", type);
   }
-
-  ArrayBuilder* builder() const { return builder_; }
-
-  int num_chunks() const { return static_cast<int>(chunks_.size()); }
-
- protected:
-  ArrayBuilder* builder_;
-  bool unfinished_builder_;
-  std::vector<std::shared_ptr<Array>> chunks_;
 };
 
-template <typename Type, NullCoding null_coding = NullCoding::NONE_ONLY>
-class TypedConverter : public SeqConverter {
- public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  Status Init(ArrayBuilder* builder) override {
-    builder_ = builder;
-    DCHECK_NE(builder_, nullptr);
-    typed_builder_ = checked_cast<BuilderType*>(builder);
-    return Status::OK();
-  }
-
-  // Append a missing item (default implementation)
-  Status AppendNull() override { return this->typed_builder_->AppendNull(); }
-
-  // Append null if the obj is None or pandas null otherwise the valid value
-  Status Append(PyObject* obj) override {
-    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
-  }
-
-  Status Extend(PyObject* obj, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequence(
-        obj, [this](PyObject* item, bool* /* unused */) { return this->Append(item); });
-  }
-
-  Status ExtendMasked(PyObject* obj, PyObject* mask, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequenceMasked(
-        obj, mask, [this](PyObject* item, bool is_masked, bool* /* unused */) {
-          if (is_masked) {
-            return this->AppendNull();
-          } else {
-            // This will also apply the null-checking convention in the event
-            // that the value is not masked
-            return this->Append(item);  // perhaps use AppendValue instead?
-          }
-        });
-  }
+template <typename T>
+Status Extend(T* converter, PyObject* values, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequence(values, [converter](PyObject* item, bool* /* unused */) {
+    return converter->Append(item);
+  });
+}
 
- protected:
-  BuilderType* typed_builder_;
-};
+// Convert and append a sequence of values masked with a numpy array
+template <typename T>
+Status ExtendMasked(T* converter, PyObject* values, PyObject* mask, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequenceMasked(
+      values, mask, [converter](PyObject* item, bool is_masked, bool* /* unused */) {
+        if (is_masked) {
+          return converter->AppendNull();
+        } else {
+          // This will also apply the null-checking convention in the event
+          // that the value is not masked
+          return converter->Append(item);  // perhaps use AppendValue instead?
+        }
+      });
+}
 
-// ----------------------------------------------------------------------
-// Sequence converter for null type
+// The base Converter class is a mixin with predefined behavior and constructors.
+using PyConverter = Converter<PyObject*, PyConversionOptions>;
 
-template <NullCoding null_coding>
-class NullConverter : public TypedConverter<NullType, null_coding> {
- public:
-  Status AppendValue(PyObject* obj) override {
-    return internal::InvalidValue(obj, "converting to null type");
-  }
-};
+template <typename T, typename Enable = void>
+class PyPrimitiveConverter;
 
-// ----------------------------------------------------------------------
-// Sequence converter template for primitive (integer and floating point bool) types
+template <typename T>
+class PyListConverter;
 
-template <typename Type, NullCoding null_coding>
-class PrimitiveConverter : public TypedConverter<Type, null_coding> {
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value, ValueConverter<Type>::FromPython(obj));
-    return this->typed_builder_->Append(value);
-  }
-};
+template <typename U, typename Enable = void>
+class PyDictionaryConverter;
 
-// ----------------------------------------------------------------------
-// Sequence converters for temporal types
+class PyStructConverter;
 
-template <typename Type, NullCoding null_coding>
-class TimeConverter : public TypedConverter<Type, null_coding> {
- public:
-  explicit TimeConverter(TimeUnit::type unit, bool ignore_timezone)
-      : unit_(unit), ignore_timezone_(ignore_timezone) {}
-
-  // TODO(kszucs): support numpy values for date and time converters
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value,
-                          ValueConverter<Type>::FromPython(obj, unit_, ignore_timezone_));
-    return this->typed_builder_->Append(value);
-  }
+template <typename T, typename Enable = void>
+struct PyConverterTrait;
 
- protected:
-  TimeUnit::type unit_;
-  bool ignore_timezone_;
+template <typename T>
+struct PyConverterTrait<
+    T, enable_if_t<!is_nested_type<T>::value && !is_interval_type<T>::value &&
+                   !is_extension_type<T>::value>> {
+  using type = PyPrimitiveConverter<T>;
 };
 
-// TODO(kszucs): move it to the type_traits
 template <typename T>
-struct NumpyType {};
+struct PyConverterTrait<T, enable_if_list_like<T>> {
+  using type = PyListConverter<T>;
+};
 
 template <>
-struct NumpyType<TimestampType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_DATETIME>::isnull(v);
-  }
+struct PyConverterTrait<StructType> {
+  using type = PyStructConverter;
 };
 
 template <>
-struct NumpyType<DurationType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_TIMEDELTA>::isnull(v);
-  }
+struct PyConverterTrait<DictionaryType> {
+  template <typename T>
+  using type = PyDictionaryConverter<T>;
 };
 
-template <typename Type, NullCoding null_coding>
-class TemporalConverter : public TimeConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_null_type<T>::value || is_boolean_type<T>::value ||
+                   is_number_type<T>::value || is_decimal_type<T>::value ||
+                   is_date_type<T>::value || is_time_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using TimeConverter<Type, null_coding>::TimeConverter;
-
-  Status AppendValue(PyObject* obj) override {
-    int64_t value;
-    if (PyArray_CheckAnyScalarExact(obj)) {
-      // convert np.datetime64 / np.timedelta64 depending on Type
-      ARROW_ASSIGN_OR_RAISE(value, ValueConverter<Type>::FromNumpy(obj, this->unit_));
-      if (NumpyType<Type>::isnull(value)) {
-        // checks numpy NaT sentinel after conversion
-        return this->typed_builder_->AppendNull();
-      }
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
     } else {
       ARROW_ASSIGN_OR_RAISE(
-          value,
-          ValueConverter<Type>::FromPython(
-              obj, this->unit_, TimeConverter<Type, null_coding>::ignore_timezone_));
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      return this->primitive_builder_->Append(converted);
     }
-    return this->typed_builder_->Append(value);
   }
 };
 
-// ----------------------------------------------------------------------
-// Sequence converters for Binary, FixedSizeBinary, String
-
-template <typename Type, NullCoding null_coding>
-class BinaryLikeConverter : public TypedConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_timestamp_type<T>::value || is_duration_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  inline Status AutoChunk(Py_ssize_t size) {
-    // did we reach the builder size limit?
-    if (ARROW_PREDICT_FALSE(this->typed_builder_->value_data_length() + size >
-                            BuilderType::memory_limit())) {
-      // builder would be full, so need to add a new chunk
-      std::shared_ptr<Array> chunk;
-      RETURN_NOT_OK(this->typed_builder_->Finish(&chunk));
-      this->chunks_.emplace_back(std::move(chunk));
-    }
-    return Status::OK();
-  }
-
-  Status AppendString(const PyBytesView& view) {
-    // check that the value fits in the datatype
-    if (view.size > BuilderType::memory_limit()) {
-      return Status::Invalid("string too large for datatype");
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      // Numpy NaT sentinels can be checked after the conversion
+      if (PyArray_CheckAnyScalarExact(value) &&
+          PyValue::IsNaT(this->primitive_type_, converted)) {
+        return this->primitive_builder_->AppendNull();
+      } else {
+        return this->primitive_builder_->Append(converted);
+      }
     }
-    DCHECK_GE(view.size, 0);
-
-    // create a new chunk if the value would overflow the builder
-    RETURN_NOT_OK(AutoChunk(view.size));
-
-    // now we can safely append the value to the builder
-    RETURN_NOT_OK(
-        this->typed_builder_->Append(::arrow::util::string_view(view.bytes, view.size)));
-
-    return Status::OK();
   }
-
- protected:
-  // Create a single instance of PyBytesView here to prevent unnecessary object
-  // creation/destruction
-  PyBytesView string_view_;
 };
 
-template <typename Type, NullCoding null_coding>
-class BinaryConverter : public BinaryLikeConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_binary<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto view, ValueConverter<Type>::FromPython(obj));
-    return this->AppendString(view);
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
+    }
   }
 };
 
-template <NullCoding null_coding>
-class FixedSizeBinaryConverter
-    : public BinaryLikeConverter<FixedSizeBinaryType, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_string_like<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  explicit FixedSizeBinaryConverter(int32_t byte_width) : byte_width_(byte_width) {}
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      if (!view.is_utf8) {
+        // observed binary value
+        observed_binary_ = true;
+      }
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
+    }
+  }
 
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(
-        this->string_view_,
-        ValueConverter<FixedSizeBinaryType>::FromPython(obj, byte_width_));
-    return this->AppendString(this->string_view_);
+  Result<std::shared_ptr<Array>> ToArray() override {
+    ARROW_ASSIGN_OR_RAISE(auto array, (PrimitiveConverter<T, PyConverter>::ToArray()));
+    if (observed_binary_) {
+      // if we saw any non-unicode, cast results to BinaryArray
+      auto binary_type = TypeTraits<typename T::PhysicalType>::type_singleton();
+      return array->View(binary_type);
+    } else {
+      return array;
+    }
   }
 
  protected:
-  int32_t byte_width_;
+  bool observed_binary_ = false;
 };
 
-// For String/UTF8, if strict_conversions enabled, we reject any non-UTF8,
-// otherwise we allow but return results as BinaryArray
-template <typename Type, bool Strict, NullCoding null_coding>
-class StringConverter : public BinaryLikeConverter<Type, null_coding> {
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_c_type<U>>
+    : public DictionaryConverter<U, PyConverter> {
  public:
-  StringConverter() : binary_count_(0) {}
-
-  Status AppendValue(PyObject* obj) override {
-    if (Strict) {
-      // raise if the object is not unicode or not an utf-8 encoded bytes
-      ARROW_ASSIGN_OR_RAISE(this->string_view_, ValueConverter<Type>::FromPython(obj));
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
     } else {
-      // keep track of whether values are unicode or bytes; if any bytes are
-      // observe, the result will be bytes
-      bool is_utf8;
-      ARROW_ASSIGN_OR_RAISE(this->string_view_,
-                            ValueConverter<Type>::FromPython(obj, &is_utf8));
-      if (!is_utf8) {
-        ++binary_count_;
-      }
+      ARROW_ASSIGN_OR_RAISE(auto converted,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(converted);
     }
-    return this->AppendString(this->string_view_);
   }
+};
 
-  Status GetResult(std::shared_ptr<ChunkedArray>* out) override {
-    RETURN_NOT_OK(SeqConverter::GetResult(out));
-
-    // If we saw any non-unicode, cast results to BinaryArray
-    if (binary_count_) {
-      // We should have bailed out earlier
-      DCHECK(!Strict);
-      auto binary_type = TypeTraits<typename Type::PhysicalType>::type_singleton();
-      return (*out)->View(binary_type).Value(out);
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_string_view<U>>
+    : public DictionaryConverter<U, PyConverter> {
+ public:
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(auto view,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(util::string_view(view.bytes, view.size));
     }
-    return Status::OK();
   }
-
- protected:
-  int64_t binary_count_;
 };
 
-// ----------------------------------------------------------------------
-// Convert lists (NumPy arrays containing lists or ndarrays as values)
-
 // If the value type does not match the expected NumPy dtype, then fall through
 // to a slower PySequence-based path
-#define LIST_FAST_CASE(TYPE, NUMPY_TYPE, ArrowType)            \
-  case Type::TYPE: {                                           \
-    if (PyArray_DESCR(arr)->type_num != NUMPY_TYPE) {          \
-      return value_converter_->Extend(obj, value_length);      \
-    }                                                          \
-    return AppendNdarrayTypedItem<NUMPY_TYPE, ArrowType>(arr); \
+#define LIST_FAST_CASE(TYPE_ID, TYPE, NUMPY_TYPE)               \
+  case Type::TYPE_ID: {                                         \
+    if (PyArray_DESCR(ndarray)->type_num != NUMPY_TYPE) {       \
+      return Extend(this->value_converter_.get(), value, size); \
+    }                                                           \
+    return AppendNdarrayTyped<TYPE, NUMPY_TYPE>(ndarray);       \
   }
 
 // Use internal::VisitSequence, fast for NPY_OBJECT but slower otherwise
-#define LIST_SLOW_CASE(TYPE)                            \
-  case Type::TYPE: {                                    \
-    return value_converter_->Extend(obj, value_length); \
+#define LIST_SLOW_CASE(TYPE_ID)                               \
+  case Type::TYPE_ID: {                                       \
+    return Extend(this->value_converter_.get(), value, size); \
   }
 
-// Base class for ListConverter and FixedSizeListConverter (to have both work with CRTP)
-template <typename TypeClass, NullCoding null_coding>
-class BaseListConverter : public TypedConverter<TypeClass, null_coding> {
+template <typename T>
+class PyListConverter : public ListConverter<T, PyConverter, PyConverterTrait> {
  public:
-  using BuilderType = typename TypeTraits<TypeClass>::BuilderType;
-
-  explicit BaseListConverter(bool from_pandas, bool strict_conversions,
-                             bool ignore_timezone)
-      : from_pandas_(from_pandas),
-        strict_conversions_(strict_conversions),
-        ignore_timezone_(ignore_timezone) {}
-
-  Status Init(ArrayBuilder* builder) override {
-    this->builder_ = builder;
-    this->typed_builder_ = checked_cast<BuilderType*>(builder);
-
-    this->value_type_ = checked_cast<const TypeClass&>(*builder->type()).value_type();
-    RETURN_NOT_OK(GetConverter(value_type_, from_pandas_, strict_conversions_,
-                               ignore_timezone_, &value_converter_));
-    return this->value_converter_->Init(this->typed_builder_->value_builder());
-  }
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->list_builder_->AppendNull();
+    }
 
-  template <int NUMPY_TYPE, typename Type>
-  Status AppendNdarrayTypedItem(PyArrayObject* arr) {
-    using traits = internal::npy_traits<NUMPY_TYPE>;
-    using T = typename traits::value_type;
-    using ValueBuilderType = typename TypeTraits<Type>::BuilderType;
+    RETURN_NOT_OK(this->list_builder_->Append());
+    if (PyArray_Check(value)) {
+      RETURN_NOT_OK(AppendNdarray(value));
+    } else if (PySequence_Check(value)) {
+      RETURN_NOT_OK(AppendSequence(value));
+    } else {
+      return internal::InvalidType(
+          value, "was not a sequence or recognized null for conversion to list type");
+    }
 
-    const bool null_sentinels_possible =
-        // Always treat Numpy's NaT as null
-        NUMPY_TYPE == NPY_DATETIME || NUMPY_TYPE == NPY_TIMEDELTA ||
-        // Observing pandas's null sentinels
-        (from_pandas_ && traits::supports_nulls);
+    return ValidateBuilder(this->list_type_);
+  }
 
-    auto child_builder = checked_cast<ValueBuilderType*>(value_converter_->builder());
+ protected:
+  Status ValidateOverflow(const MapType*, int64_t size) { return Status::OK(); }

Review comment:
       Added a test case for it but takes 20 minutes to run. 

##########
File path: python/pyarrow/array.pxi
##########
@@ -21,28 +21,28 @@ import warnings
 
 cdef _sequence_to_array(object sequence, object mask, object size,
                         DataType type, CMemoryPool* pool, c_bool from_pandas):
-    cdef int64_t c_size
-    cdef PyConversionOptions options
+    cdef:
+        int64_t c_size
+        PyConversionOptions options
+        shared_ptr[CChunkedArray] chunked
 
     if type is not None:
         options.type = type.sp_type
 
     if size is not None:
         options.size = size
 
-    options.pool = pool
     options.from_pandas = from_pandas
-    options.ignore_timezone = os.environ.get('PYARROW_IGNORE_TIMEZONE', False)

Review comment:
       Added a [unittest](https://github.com/apache/arrow/pull/8088/commits/29aa420f020233f29fd8ff225c58717e6c17b634#diff-7fddb4a3590947db6d84787eca4d80faR969-R998) for it, please check its validity.

##########
File path: python/pyarrow/tests/test_convert_builtin.py
##########
@@ -132,6 +133,10 @@ def _as_tuple(xs):
     return tuple(xs)
 
 
+def _as_pairs(xs):

Review comment:
       Not used anymore.




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter>
+struct MakeConverterImpl;
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  template <typename T>
+  using Primitive = PrimitiveConverter<T, Self>;
+  template <typename T>
+  using List = ListConverter<T, Self>;
+  template <typename T>
+  using Dictionary = DictionaryConverter<T, Self>;
+  using Struct = StructConverter<Self>;
+
+  static Result<std::shared_ptr<Self>> Make(std::shared_ptr<DataType> type,
+                                            MemoryPool* pool, OptionsType options) {
+    std::shared_ptr<Self> out;
+    MakeConverterImpl<Self> visitor = {type, pool, options, &out};
+    ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+    ARROW_RETURN_NOT_OK(out->Init());
+    return out;
+  }
+
+  virtual ~Converter() = default;
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>> children() const { return children_; }
+
+  virtual Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  virtual Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  friend struct MakeConverterImpl<Self>;
+
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+#define DICTIONARY_CASE(TYPE_ENUM, TYPE_CLASS)                          \
+  case Type::TYPE_ENUM:                                                 \
+    return Finish<typename Converter::template Dictionary<TYPE_CLASS>>( \
+        std::move(builder), {});                                        \
+    break;
+
+template <typename Converter>
+struct MakeConverterImpl {
+  Status Visit(const NullType& t) {
+    using BuilderType = typename TypeTraits<NullType>::BuilderType;
+    using ConverterType = typename Converter::template Primitive<NullType>;

Review comment:
       Added the following, the dictionary type required an additional type parameter:
   
   ```cpp
   template <typename T, typename Enable = void>
   struct PyConverterTrait;
   
   template <typename T>
   struct PyConverterTrait<T, enable_if_not_nested<T>> {
     using type = PyPrimitiveConverter<T>;
   };
   
   template <typename T>
   struct PyConverterTrait<T, enable_if_list_like<T>> {
     using type = PyListConverter<T>;
   };
   
   template <>
   struct PyConverterTrait<StructType> {
     using type = PyStructConverter;
   };
   
   template <>
   struct PyConverterTrait<DictionaryType> {
     template <typename T>
     using type = PyDictionaryConverter<T>;
   };
   ```




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

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



[GitHub] [arrow] kszucs commented on pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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


   @github-actions crossbow submit -g wheel -g integration


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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/array.pxi
##########
@@ -21,28 +21,28 @@ import warnings
 
 cdef _sequence_to_array(object sequence, object mask, object size,
                         DataType type, CMemoryPool* pool, c_bool from_pandas):
-    cdef int64_t c_size
-    cdef PyConversionOptions options
+    cdef:
+        int64_t c_size
+        PyConversionOptions options
+        shared_ptr[CChunkedArray] chunked
 
     if type is not None:
         options.type = type.sp_type
 
     if size is not None:
         options.size = size
 
-    options.pool = pool
     options.from_pandas = from_pandas
-    options.ignore_timezone = os.environ.get('PYARROW_IGNORE_TIMEZONE', False)

Review comment:
       It wasn't catched by any 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.

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



[GitHub] [arrow] github-actions[bot] commented on pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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


   Revision: ecf974670e78e86a47511e3293d192b9f9dfbf2f
   
   Submitted crossbow builds: [ursa-labs/crossbow @ actions-551](https://github.com/ursa-labs/crossbow/branches/all?query=actions-551)
   
   |Task|Status|
   |----|------|
   |test-conda-python-3.6-pandas-0.23|[![Github Actions](https://github.com/ursa-labs/crossbow/workflows/Crossbow/badge.svg?branch=actions-551-github-test-conda-python-3.6-pandas-0.23)](https://github.com/ursa-labs/crossbow/actions?query=branch:actions-551-github-test-conda-python-3.6-pandas-0.23)|
   |test-conda-python-3.7-dask-latest|[![Github Actions](https://github.com/ursa-labs/crossbow/workflows/Crossbow/badge.svg?branch=actions-551-github-test-conda-python-3.7-dask-latest)](https://github.com/ursa-labs/crossbow/actions?query=branch:actions-551-github-test-conda-python-3.7-dask-latest)|
   |test-conda-python-3.7-hdfs-2.9.2|[![Github Actions](https://github.com/ursa-labs/crossbow/workflows/Crossbow/badge.svg?branch=actions-551-github-test-conda-python-3.7-hdfs-2.9.2)](https://github.com/ursa-labs/crossbow/actions?query=branch:actions-551-github-test-conda-python-3.7-hdfs-2.9.2)|
   |test-conda-python-3.7-kartothek-latest|[![Github Actions](https://github.com/ursa-labs/crossbow/workflows/Crossbow/badge.svg?branch=actions-551-github-test-conda-python-3.7-kartothek-latest)](https://github.com/ursa-labs/crossbow/actions?query=branch:actions-551-github-test-conda-python-3.7-kartothek-latest)|
   |test-conda-python-3.7-kartothek-master|[![Github Actions](https://github.com/ursa-labs/crossbow/workflows/Crossbow/badge.svg?branch=actions-551-github-test-conda-python-3.7-kartothek-master)](https://github.com/ursa-labs/crossbow/actions?query=branch:actions-551-github-test-conda-python-3.7-kartothek-master)|
   |test-conda-python-3.7-pandas-latest|[![Github Actions](https://github.com/ursa-labs/crossbow/workflows/Crossbow/badge.svg?branch=actions-551-github-test-conda-python-3.7-pandas-latest)](https://github.com/ursa-labs/crossbow/actions?query=branch:actions-551-github-test-conda-python-3.7-pandas-latest)|
   |test-conda-python-3.7-pandas-master|[![Github Actions](https://github.com/ursa-labs/crossbow/workflows/Crossbow/badge.svg?branch=actions-551-github-test-conda-python-3.7-pandas-master)](https://github.com/ursa-labs/crossbow/actions?query=branch:actions-551-github-test-conda-python-3.7-pandas-master)|
   |test-conda-python-3.7-spark-branch-3.0|[![Github Actions](https://github.com/ursa-labs/crossbow/workflows/Crossbow/badge.svg?branch=actions-551-github-test-conda-python-3.7-spark-branch-3.0)](https://github.com/ursa-labs/crossbow/actions?query=branch:actions-551-github-test-conda-python-3.7-spark-branch-3.0)|
   |test-conda-python-3.7-turbodbc-latest|[![Github Actions](https://github.com/ursa-labs/crossbow/workflows/Crossbow/badge.svg?branch=actions-551-github-test-conda-python-3.7-turbodbc-latest)](https://github.com/ursa-labs/crossbow/actions?query=branch:actions-551-github-test-conda-python-3.7-turbodbc-latest)|
   |test-conda-python-3.7-turbodbc-master|[![Github Actions](https://github.com/ursa-labs/crossbow/workflows/Crossbow/badge.svg?branch=actions-551-github-test-conda-python-3.7-turbodbc-master)](https://github.com/ursa-labs/crossbow/actions?query=branch:actions-551-github-test-conda-python-3.7-turbodbc-master)|
   |test-conda-python-3.8-dask-master|[![Github Actions](https://github.com/ursa-labs/crossbow/workflows/Crossbow/badge.svg?branch=actions-551-github-test-conda-python-3.8-dask-master)](https://github.com/ursa-labs/crossbow/actions?query=branch:actions-551-github-test-conda-python-3.8-dask-master)|
   |test-conda-python-3.8-jpype|[![Github Actions](https://github.com/ursa-labs/crossbow/workflows/Crossbow/badge.svg?branch=actions-551-github-test-conda-python-3.8-jpype)](https://github.com/ursa-labs/crossbow/actions?query=branch:actions-551-github-test-conda-python-3.8-jpype)|
   |test-conda-python-3.8-pandas-latest|[![Github Actions](https://github.com/ursa-labs/crossbow/workflows/Crossbow/badge.svg?branch=actions-551-github-test-conda-python-3.8-pandas-latest)](https://github.com/ursa-labs/crossbow/actions?query=branch:actions-551-github-test-conda-python-3.8-pandas-latest)|
   |test-conda-python-3.8-spark-master|[![Github Actions](https://github.com/ursa-labs/crossbow/workflows/Crossbow/badge.svg?branch=actions-551-github-test-conda-python-3.8-spark-master)](https://github.com/ursa-labs/crossbow/actions?query=branch:actions-551-github-test-conda-python-3.8-spark-master)|
   |wheel-manylinux1-cp35m|[![Azure](https://dev.azure.com/ursa-labs/crossbow/_apis/build/status/ursa-labs.crossbow?branchName=actions-551-azure-wheel-manylinux1-cp35m)](https://dev.azure.com/ursa-labs/crossbow/_build/latest?definitionId=1&branchName=actions-551-azure-wheel-manylinux1-cp35m)|
   |wheel-manylinux1-cp36m|[![Azure](https://dev.azure.com/ursa-labs/crossbow/_apis/build/status/ursa-labs.crossbow?branchName=actions-551-azure-wheel-manylinux1-cp36m)](https://dev.azure.com/ursa-labs/crossbow/_build/latest?definitionId=1&branchName=actions-551-azure-wheel-manylinux1-cp36m)|
   |wheel-manylinux1-cp37m|[![Azure](https://dev.azure.com/ursa-labs/crossbow/_apis/build/status/ursa-labs.crossbow?branchName=actions-551-azure-wheel-manylinux1-cp37m)](https://dev.azure.com/ursa-labs/crossbow/_build/latest?definitionId=1&branchName=actions-551-azure-wheel-manylinux1-cp37m)|
   |wheel-manylinux1-cp38|[![Azure](https://dev.azure.com/ursa-labs/crossbow/_apis/build/status/ursa-labs.crossbow?branchName=actions-551-azure-wheel-manylinux1-cp38)](https://dev.azure.com/ursa-labs/crossbow/_build/latest?definitionId=1&branchName=actions-551-azure-wheel-manylinux1-cp38)|
   |wheel-manylinux2010-cp35m|[![Azure](https://dev.azure.com/ursa-labs/crossbow/_apis/build/status/ursa-labs.crossbow?branchName=actions-551-azure-wheel-manylinux2010-cp35m)](https://dev.azure.com/ursa-labs/crossbow/_build/latest?definitionId=1&branchName=actions-551-azure-wheel-manylinux2010-cp35m)|
   |wheel-manylinux2010-cp36m|[![Azure](https://dev.azure.com/ursa-labs/crossbow/_apis/build/status/ursa-labs.crossbow?branchName=actions-551-azure-wheel-manylinux2010-cp36m)](https://dev.azure.com/ursa-labs/crossbow/_build/latest?definitionId=1&branchName=actions-551-azure-wheel-manylinux2010-cp36m)|
   |wheel-manylinux2010-cp37m|[![Azure](https://dev.azure.com/ursa-labs/crossbow/_apis/build/status/ursa-labs.crossbow?branchName=actions-551-azure-wheel-manylinux2010-cp37m)](https://dev.azure.com/ursa-labs/crossbow/_build/latest?definitionId=1&branchName=actions-551-azure-wheel-manylinux2010-cp37m)|
   |wheel-manylinux2010-cp38|[![Azure](https://dev.azure.com/ursa-labs/crossbow/_apis/build/status/ursa-labs.crossbow?branchName=actions-551-azure-wheel-manylinux2010-cp38)](https://dev.azure.com/ursa-labs/crossbow/_build/latest?definitionId=1&branchName=actions-551-azure-wheel-manylinux2010-cp38)|
   |wheel-manylinux2014-cp35m|[![Azure](https://dev.azure.com/ursa-labs/crossbow/_apis/build/status/ursa-labs.crossbow?branchName=actions-551-azure-wheel-manylinux2014-cp35m)](https://dev.azure.com/ursa-labs/crossbow/_build/latest?definitionId=1&branchName=actions-551-azure-wheel-manylinux2014-cp35m)|
   |wheel-manylinux2014-cp36m|[![Azure](https://dev.azure.com/ursa-labs/crossbow/_apis/build/status/ursa-labs.crossbow?branchName=actions-551-azure-wheel-manylinux2014-cp36m)](https://dev.azure.com/ursa-labs/crossbow/_build/latest?definitionId=1&branchName=actions-551-azure-wheel-manylinux2014-cp36m)|
   |wheel-manylinux2014-cp37m|[![Azure](https://dev.azure.com/ursa-labs/crossbow/_apis/build/status/ursa-labs.crossbow?branchName=actions-551-azure-wheel-manylinux2014-cp37m)](https://dev.azure.com/ursa-labs/crossbow/_build/latest?definitionId=1&branchName=actions-551-azure-wheel-manylinux2014-cp37m)|
   |wheel-manylinux2014-cp38|[![Azure](https://dev.azure.com/ursa-labs/crossbow/_apis/build/status/ursa-labs.crossbow?branchName=actions-551-azure-wheel-manylinux2014-cp38)](https://dev.azure.com/ursa-labs/crossbow/_build/latest?definitionId=1&branchName=actions-551-azure-wheel-manylinux2014-cp38)|
   |wheel-osx-cp35m|[![TravisCI](https://img.shields.io/travis/ursa-labs/crossbow/actions-551-travis-wheel-osx-cp35m.svg)](https://travis-ci.org/ursa-labs/crossbow/branches)|
   |wheel-osx-cp36m|[![TravisCI](https://img.shields.io/travis/ursa-labs/crossbow/actions-551-travis-wheel-osx-cp36m.svg)](https://travis-ci.org/ursa-labs/crossbow/branches)|
   |wheel-osx-cp37m|[![TravisCI](https://img.shields.io/travis/ursa-labs/crossbow/actions-551-travis-wheel-osx-cp37m.svg)](https://travis-ci.org/ursa-labs/crossbow/branches)|
   |wheel-osx-cp38|[![TravisCI](https://img.shields.io/travis/ursa-labs/crossbow/actions-551-travis-wheel-osx-cp38.svg)](https://travis-ci.org/ursa-labs/crossbow/branches)|
   |wheel-win-cp35m|[![Appveyor](https://img.shields.io/appveyor/ci/ursa-labs/crossbow/actions-551-appveyor-wheel-win-cp35m.svg)](https://ci.appveyor.com/project/ursa-labs/crossbow/history)|
   |wheel-win-cp36m|[![Appveyor](https://img.shields.io/appveyor/ci/ursa-labs/crossbow/actions-551-appveyor-wheel-win-cp36m.svg)](https://ci.appveyor.com/project/ursa-labs/crossbow/history)|
   |wheel-win-cp37m|[![Appveyor](https://img.shields.io/appveyor/ci/ursa-labs/crossbow/actions-551-appveyor-wheel-win-cp37m.svg)](https://ci.appveyor.com/project/ursa-labs/crossbow/history)|
   |wheel-win-cp38|[![Appveyor](https://img.shields.io/appveyor/ci/ursa-labs/crossbow/actions-551-appveyor-wheel-win-cp38.svg)](https://ci.appveyor.com/project/ursa-labs/crossbow/history)|


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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter>
+struct MakeConverterImpl;
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  template <typename T>
+  using Primitive = PrimitiveConverter<T, Self>;
+  template <typename T>
+  using List = ListConverter<T, Self>;
+  template <typename T>
+  using Dictionary = DictionaryConverter<T, Self>;
+  using Struct = StructConverter<Self>;
+
+  static Result<std::shared_ptr<Self>> Make(std::shared_ptr<DataType> type,
+                                            MemoryPool* pool, OptionsType options) {
+    std::shared_ptr<Self> out;
+    MakeConverterImpl<Self> visitor = {type, pool, options, &out};
+    ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+    ARROW_RETURN_NOT_OK(out->Init());
+    return out;
+  }
+
+  virtual ~Converter() = default;
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>> children() const { return children_; }
+
+  virtual Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  virtual Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }

Review comment:
       It is overridden from one of the python converters to change the type of the output array based on observed values.




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -1352,64 +927,40 @@ Status ConvertToSequenceAndInferSize(PyObject* obj, PyObject** seq, int64_t* siz
   return Status::OK();
 }
 
-Status ConvertPySequence(PyObject* sequence_source, PyObject* mask,
-                         const PyConversionOptions& options,
-                         std::shared_ptr<ChunkedArray>* out) {
+Result<std::shared_ptr<ChunkedArray>> ConvertPySequence(PyObject* obj, PyObject* mask,
+                                                        const PyConversionOptions& opts,
+                                                        MemoryPool* pool) {
   PyAcquireGIL lock;
 
   PyObject* seq;
   OwnedRef tmp_seq_nanny;
-
-  std::shared_ptr<DataType> real_type;
+  PyConversionOptions options = opts;  // copy options struct since we modify it below

Review comment:
       Right, updated.




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/array.pxi
##########
@@ -21,28 +21,28 @@ import warnings
 
 cdef _sequence_to_array(object sequence, object mask, object size,
                         DataType type, CMemoryPool* pool, c_bool from_pandas):
-    cdef int64_t c_size
-    cdef PyConversionOptions options
+    cdef:
+        int64_t c_size
+        PyConversionOptions options
+        shared_ptr[CChunkedArray] chunked
 
     if type is not None:
         options.type = type.sp_type
 
     if size is not None:
         options.size = size
 
-    options.pool = pool
     options.from_pandas = from_pandas
-    options.ignore_timezone = os.environ.get('PYARROW_IGNORE_TIMEZONE', False)

Review comment:
       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.

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>

Review comment:
       Updated.




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter>
+struct MakeConverterImpl;
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  template <typename T>
+  using Primitive = PrimitiveConverter<T, Self>;
+  template <typename T>
+  using List = ListConverter<T, Self>;
+  template <typename T>
+  using Dictionary = DictionaryConverter<T, Self>;
+  using Struct = StructConverter<Self>;
+
+  static Result<std::shared_ptr<Self>> Make(std::shared_ptr<DataType> type,
+                                            MemoryPool* pool, OptionsType options) {
+    std::shared_ptr<Self> out;
+    MakeConverterImpl<Self> visitor = {type, pool, options, &out};
+    ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+    ARROW_RETURN_NOT_OK(out->Init());
+    return out;
+  }
+
+  virtual ~Converter() = default;
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>> children() const { return children_; }
+
+  virtual Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  virtual Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  friend struct MakeConverterImpl<Self>;
+
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+#define DICTIONARY_CASE(TYPE_ENUM, TYPE_CLASS)                          \
+  case Type::TYPE_ENUM:                                                 \
+    return Finish<typename Converter::template Dictionary<TYPE_CLASS>>( \
+        std::move(builder), {});                                        \
+    break;
+
+template <typename Converter>
+struct MakeConverterImpl {
+  Status Visit(const NullType& t) {
+    using BuilderType = typename TypeTraits<NullType>::BuilderType;
+    using ConverterType = typename Converter::template Primitive<NullType>;
+
+    auto builder = std::make_shared<BuilderType>(pool);
+    return Finish<ConverterType>(std::move(builder), {});
+  }
+
+  template <typename T>
+  enable_if_t<!is_nested_type<T>::value && !is_interval_type<T>::value &&
+                  !is_dictionary_type<T>::value && !is_extension_type<T>::value,
+              Status>
+  Visit(const T& t) {
+    using BuilderType = typename TypeTraits<T>::BuilderType;
+    using ConverterType = typename Converter::template Primitive<T>;
+
+    auto builder = std::make_shared<BuilderType>(type, pool);
+    return Finish<ConverterType>(std::move(builder), {});
+  }
+
+  template <typename T>
+  enable_if_t<is_list_like_type<T>::value && !std::is_same<T, MapType>::value, Status>
+  Visit(const T& t) {
+    using BuilderType = typename TypeTraits<T>::BuilderType;
+    using ConverterType = typename Converter::template List<T>;
+
+    ARROW_ASSIGN_OR_RAISE(auto child_converter,
+                          Converter::Make(t.value_type(), pool, options));
+    auto builder = std::make_shared<BuilderType>(pool, child_converter->builder(), type);
+    return Finish<ConverterType>(std::move(builder), {std::move(child_converter)});
+  }
+
+  Status Visit(const MapType& t) {
+    using ConverterType = typename Converter::template List<MapType>;
+
+    // TODO(kszucs): seems like builders not respect field nullability
+    std::vector<std::shared_ptr<Field>> struct_fields{t.key_field(), t.item_field()};
+    auto struct_type = std::make_shared<StructType>(struct_fields);
+    ARROW_ASSIGN_OR_RAISE(auto struct_converter,
+                          Converter::Make(struct_type, pool, options));
+
+    auto struct_builder = struct_converter->builder();
+    auto key_builder = struct_builder->child_builder(0);
+    auto item_builder = struct_builder->child_builder(1);
+    auto builder = std::make_shared<MapBuilder>(pool, key_builder, item_builder, type);
+
+    return Finish<ConverterType>(std::move(builder), {std::move(struct_converter)});
+  }
+
+  Status Visit(const DictionaryType& t) {
+    std::unique_ptr<ArrayBuilder> builder;
+    ARROW_RETURN_NOT_OK(MakeDictionaryBuilder(pool, type, NULLPTR, &builder));
+
+    switch (t.value_type()->id()) {
+      DICTIONARY_CASE(BOOL, BooleanType);
+      DICTIONARY_CASE(INT8, Int8Type);
+      DICTIONARY_CASE(INT16, Int16Type);
+      DICTIONARY_CASE(INT32, Int32Type);
+      DICTIONARY_CASE(INT64, Int64Type);
+      DICTIONARY_CASE(UINT8, UInt8Type);
+      DICTIONARY_CASE(UINT16, UInt16Type);
+      DICTIONARY_CASE(UINT32, UInt32Type);
+      DICTIONARY_CASE(UINT64, UInt64Type);
+      DICTIONARY_CASE(HALF_FLOAT, HalfFloatType);
+      DICTIONARY_CASE(FLOAT, FloatType);
+      DICTIONARY_CASE(DOUBLE, DoubleType);
+      DICTIONARY_CASE(DATE32, Date32Type);
+      DICTIONARY_CASE(DATE64, Date64Type);
+      DICTIONARY_CASE(BINARY, BinaryType);
+      DICTIONARY_CASE(STRING, StringType);
+      DICTIONARY_CASE(FIXED_SIZE_BINARY, FixedSizeBinaryType);
+      default:
+        return Status::NotImplemented("DictionaryArray converter for type ", t.ToString(),
+                                      " not implemented");
+    }
+  }
+
+  Status Visit(const StructType& t) {
+    using ConverterType = typename Converter::Struct;
+
+    std::shared_ptr<Converter> child_converter;
+    std::vector<std::shared_ptr<Converter>> child_converters;
+    std::vector<std::shared_ptr<ArrayBuilder>> child_builders;
+
+    for (const auto& field : t.fields()) {
+      ARROW_ASSIGN_OR_RAISE(child_converter,
+                            Converter::Make(field->type(), pool, options));
+
+      // TODO: use move
+      child_converters.push_back(child_converter);
+      child_builders.push_back(child_converter->builder());
+    }
+
+    auto builder = std::make_shared<StructBuilder>(type, pool, child_builders);
+    return Finish<ConverterType>(std::move(builder), std::move(child_converters));
+  }
+
+  Status Visit(const DataType& t) { return Status::NotImplemented(t.name()); }
+
+  template <typename ConverterType>
+  Status Finish(std::shared_ptr<ArrayBuilder> builder,
+                std::vector<std::shared_ptr<Converter>> children) {
+    auto converter = new ConverterType();
+    converter->type_ = std::move(type);
+    converter->builder_ = std::move(builder);
+    converter->options_ = options;
+    converter->children_ = std::move(children);
+    out->reset(converter);
+    return Status::OK();
+  }
+
+  const std::shared_ptr<DataType> type;
+  MemoryPool* pool;
+  typename Converter::OptionsType options;
+  std::shared_ptr<Converter>* out;
+};
+
+// TODO(kszucs): rename to AutoChunker
+template <typename BaseConverter>
+class Chunker : public BaseConverter {

Review comment:
       Agreed, updated.




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,
+                            std::shared_ptr<ArrayBuilder> builder,
+                            const std::vector<std::shared_ptr<Self>>& children,
+                            OptionsType options) {
+    type_ = std::move(type);
+    builder_ = std::move(builder);
+    children_ = std::move(children);
+    options_ = std::move(options);
+    return Init();
+  }
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>>& children() const { return children_; }
+
+  Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }

Review comment:
       Giving access to the mapping trait would require the implementor handle another template parameter. I'm updating it, but not sure that's desirable.




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,281 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename Input, typename Options>
+class ArrayConverter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  ArrayConverter(const std::shared_ptr<DataType>& type,
+                 std::shared_ptr<ArrayBuilder> builder, Options options)
+      : sp_type_(type), sp_builder_(builder), options_(options) {}
+
+  virtual ~ArrayConverter() = default;
+  const std::shared_ptr<ArrayBuilder>& builder() const { return sp_builder_; }
+  const std::shared_ptr<DataType>& type() const { return sp_type_; }
+  Options options() const { return options_; }
+
+  virtual Status Init() { return Status::OK(); }
+  virtual Status Reserve(int64_t additional_capacity) = 0;
+  virtual Status Append(InputType value) = 0;
+  virtual Status AppendNull() = 0;
+  virtual Status Extend(Input seq, int64_t size) = 0;

Review comment:
       Agreed, removing.




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

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



[GitHub] [arrow] bkietz commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter>
+struct MakeConverterImpl;
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  template <typename T>
+  using Primitive = PrimitiveConverter<T, Self>;
+  template <typename T>
+  using List = ListConverter<T, Self>;
+  template <typename T>
+  using Dictionary = DictionaryConverter<T, Self>;
+  using Struct = StructConverter<Self>;
+
+  static Result<std::shared_ptr<Self>> Make(std::shared_ptr<DataType> type,
+                                            MemoryPool* pool, OptionsType options) {
+    std::shared_ptr<Self> out;
+    MakeConverterImpl<Self> visitor = {type, pool, options, &out};
+    ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+    ARROW_RETURN_NOT_OK(out->Init());
+    return out;
+  }
+
+  virtual ~Converter() = default;
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }

Review comment:
       Implementations can still choose to return notimplemented for some types, so I think the choice to do so should be left to them




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

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



[GitHub] [arrow] github-actions[bot] removed a comment on pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

Posted by GitBox <gi...@apache.org>.
github-actions[bot] removed a comment on pull request #8088:
URL: https://github.com/apache/arrow/pull/8088#issuecomment-684114776


   <!--
     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.
   -->
   
   Thanks for opening a pull request!
   
   Could you open an issue for this pull request on JIRA?
   https://issues.apache.org/jira/browse/ARROW
   
   Then could you also rename pull request title in the following format?
   
       ARROW-${JIRA_ID}: [${COMPONENT}] ${SUMMARY}
   
   See also:
   
     * [Other pull requests](https://github.com/apache/arrow/pulls/)
     * [Contribution Guidelines - How to contribute patches](https://arrow.apache.org/docs/developers/contributing.html#how-to-contribute-patches)
   


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

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



[GitHub] [arrow] kszucs commented on pull request #8088: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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


   @romainfrancois since the conversion code in `python_to_arrow.cc` and `r/src/array_from_vector.cpp` are pretty similar it would be nice to reuse as much code as we could (including new features like auto chunking large inputs). 
   
   Could you please take a look at the base classes in [converter.h](https://github.com/apache/arrow/blob/0a384d574fde83324d4949a56220e6cda8527816/cpp/src/arrow/util/converter.h) to see whether would it help/improve the R side implementation? I have not documented it yet because it's still under discussion but you can take a look how I'm using it [in the python bindings](https://github.com/apache/arrow/blob/0a384d574fde83324d4949a56220e6cda8527816/cpp/src/arrow/python/python_to_arrow.cc#L351-L802).


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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -329,186 +300,106 @@ struct ValueConverter<DurationType> {
         default:
           return Status::UnknownError("Invalid time unit");
       }
+    } else if (PyArray_CheckAnyScalarExact(obj)) {
+      // validate that the numpy scalar has np.datetime64 dtype
+      std::shared_ptr<DataType> numpy_type;
+      RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &numpy_type));
+      if (!numpy_type->Equals(*type)) {
+        return Status::NotImplemented("Expected np.timedelta64 but got: ",
+                                      numpy_type->ToString());
+      }
+      return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
     } else {
       RETURN_NOT_OK(internal::CIntFromPython(obj, &value));
     }
     return value;
   }
 
-  static inline Result<int64_t> FromNumpy(PyObject* obj, TimeUnit::type unit) {
-    // validate that the numpy scalar has np.timedelta64 dtype
-    std::shared_ptr<DataType> type;
-    RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &type));
-    if (type->id() != DurationType::type_id) {
-      // TODO(kszucs): the message should highlight the received numpy dtype
-      return Status::Invalid("Expected np.timedelta64 but got: ", type->ToString());
-    }
-    // validate that the time units are matching
-    if (unit != checked_cast<const DurationType&>(*type).unit()) {
-      return Status::NotImplemented(
-          "Cannot convert NumPy np.timedelta64 objects with differing unit");
-    }
-    // convert the numpy value
-    return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_any_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    return std::move(view);
-  }
-};
+  // The binary-like intermediate representation is PyBytesView because it keeps temporary
+  // python objects alive (non-contiguous memoryview) and stores whether the original
+  // object was unicode encoded or not, which is used for unicode -> bytes coersion if
+  // there is a non-unicode object observed.
 
-template <typename Type>
-struct ValueConverter<Type, enable_if_string_like<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    // strict conversion, force output to be unicode / utf8 and validate that
-    // any binary values are utf8
-    bool is_utf8 = false;
-    PyBytesView view;
-
-    RETURN_NOT_OK(view.FromString(obj, &is_utf8));
-    if (!is_utf8) {
-      return internal::InvalidValue(obj, "was not a utf8 string");
-    }
-    return std::move(view);
+  static Result<PyBytesView> Convert(const BaseBinaryType*, const O&, I obj) {
+    return PyBytesView::FromString(obj);
   }
 
-  static inline Result<PyBytesView> FromPython(PyObject* obj, bool* is_utf8) {
-    PyBytesView view;
-
-    // Non-strict conversion; keep track of whether values are unicode or bytes
-    if (PyUnicode_Check(obj)) {
-      *is_utf8 = true;
-      RETURN_NOT_OK(view.FromUnicode(obj));
+  static Result<PyBytesView> Convert(const FixedSizeBinaryType* type, const O&, I obj) {
+    ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj));
+    if (ARROW_PREDICT_TRUE(view.size == type->byte_width())) {
+      return std::move(view);
     } else {
-      // If not unicode or bytes, FromBinary will error
-      *is_utf8 = false;
-      RETURN_NOT_OK(view.FromBinary(obj));
-    }
-    return std::move(view);
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_fixed_size_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj, int32_t byte_width) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    if (ARROW_PREDICT_FALSE(view.size != byte_width)) {
       std::stringstream ss;
-      ss << "expected to be length " << byte_width << " was " << view.size;
+      ss << "expected to be length " << type->byte_width() << " was " << view.size;
       return internal::InvalidValue(obj, ss.str());
-    } else {
-      return std::move(view);
     }
   }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter base and CRTP "middle" subclasses
 
-class SeqConverter;
-
-// Forward-declare converter factory
-Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
-                    bool strict_conversions, bool ignore_timezone,
-                    std::unique_ptr<SeqConverter>* out);
-
-// Marshal Python sequence (list, tuple, etc.) to Arrow array
-class SeqConverter {
- public:
-  virtual ~SeqConverter() = default;
-
-  // Initialize the sequence converter with an ArrayBuilder created
-  // externally. The reason for this interface is that we have
-  // arrow::MakeBuilder which also creates child builders for nested types, so
-  // we have to pass in the child builders to child SeqConverter in the case of
-  // converting Python objects to Arrow nested types
-  virtual Status Init(ArrayBuilder* builder) = 0;
-
-  // Append a single null value to the builder
-  virtual Status AppendNull() = 0;
-
-  // Append a valid value
-  virtual Status AppendValue(PyObject* seq) = 0;
-
-  // Append a single python object handling Null values
-  virtual Status Append(PyObject* seq) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status Extend(PyObject* seq, int64_t size) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status ExtendMasked(PyObject* seq, PyObject* mask, int64_t size) = 0;
-
-  virtual Status Close() {
-    if (chunks_.size() == 0 || builder_->length() > 0) {
-      std::shared_ptr<Array> last_chunk;
-      RETURN_NOT_OK(builder_->Finish(&last_chunk));
-      chunks_.emplace_back(std::move(last_chunk));
+  template <typename T>
+  static enable_if_string<T, Result<PyBytesView>> Convert(const T*, const O& options,
+                                                          I obj) {
+    if (options.strict) {
+      // Strict conversion, force output to be unicode / utf8 and validate that
+      // any binary values are utf8
+      ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj, true));
+      if (!view.is_utf8) {
+        return internal::InvalidValue(obj, "was not a utf8 string");
+      }
+      return std::move(view);
+    } else {
+      // Non-strict conversion; keep track of whether values are unicode or bytes
+      return PyBytesView::FromString(obj);
     }
-    return Status::OK();
   }
 
-  virtual Status GetResult(std::shared_ptr<ChunkedArray>* out) {
-    // Still some accumulated data in the builder. If there are no chunks, we
-    // always call Finish to deal with the edge case where a size-0 sequence
-    // was converted with a specific output type, like array([], type=t)
-    RETURN_NOT_OK(Close());
-    *out = std::make_shared<ChunkedArray>(this->chunks_, builder_->type());
-    return Status::OK();
+  static Result<bool> Convert(const DataType* type, const O&, I obj) {
+    return Status::NotImplemented("PyValue::Convert is not implemented for type ", type);
   }
-
-  ArrayBuilder* builder() const { return builder_; }
-
-  int num_chunks() const { return static_cast<int>(chunks_.size()); }
-
- protected:
-  ArrayBuilder* builder_;
-  bool unfinished_builder_;
-  std::vector<std::shared_ptr<Array>> chunks_;
 };
 
-template <typename Type, NullCoding null_coding = NullCoding::NONE_ONLY>
-class TypedConverter : public SeqConverter {
- public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
+// Forward-declare the type-family specific converters to inject them to the PyConverter
+// base class as type aliases.
 
-  Status Init(ArrayBuilder* builder) override {
-    builder_ = builder;
-    DCHECK_NE(builder_, nullptr);
-    typed_builder_ = checked_cast<BuilderType*>(builder);
-    return Status::OK();
-  }
+template <typename T, typename Enable = void>
+class PyPrimitiveConverter;
 
-  // Append a missing item (default implementation)
-  Status AppendNull() override { return this->typed_builder_->AppendNull(); }
+template <typename T, typename Enable = void>
+class PyDictionaryConverter;
 
-  // Append null if the obj is None or pandas null otherwise the valid value
-  Status Append(PyObject* obj) override {
-    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
-  }
+template <typename T>
+class PyListConverter;
 
-  Status Extend(PyObject* obj, int64_t size) override {
+class PyStructConverter;
+
+// The base Converter class is a mixin with predefined behavior and constructors.
+class PyConverter : public Converter<PyObject*, PyConversionOptions, PyConverter> {
+ public:
+  // Type aliases used by the parent Converter mixin's factory.
+  template <typename T>
+  using Primitive = PyPrimitiveConverter<T>;
+  template <typename T>
+  using Dictionary = PyDictionaryConverter<T>;
+  template <typename T>
+  using List = PyListConverter<T>;
+  using Struct = PyStructConverter;
+
+  // Convert and append a sequence of values
+  Status Extend(PyObject* values, int64_t size) {

Review comment:
       Updated, although I had to make these functions templated to be able to use them with the chunker as well.




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

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



[GitHub] [arrow] kszucs commented on pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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


   @github-actions crossbow submit test-*spark*


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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/array/builder_base.h
##########
@@ -56,6 +56,8 @@ class ARROW_EXPORT ArrayBuilder {
   /// skip shared pointers and just return a raw pointer
   ArrayBuilder* child(int i) { return children_[i].get(); }
 
+  std::shared_ptr<ArrayBuilder> child_builder(int i) const { return children_[i]; }

Review comment:
       Updated.




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

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



[GitHub] [arrow] wesm commented on pull request #8088: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API [WIP]

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


   Are there some related JIRAs that will be fixed by this?


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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/scalar.pxi
##########
@@ -639,12 +626,27 @@ cdef class StructScalar(Scalar, collections.abc.Mapping):
             else:
                 raise KeyError(key)
 
+    # TODO(kszucs): consider to remove __iter__, keys and values

Review comment:
       Reverted so we can find out a better solution.




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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/tests/test_convert_builtin.py
##########
@@ -1513,6 +1519,108 @@ def test_struct_from_tuples():
             pa.array([tup], type=ty)
 
 
+def test_struct_from_list_of_pairs():
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 6), ('b', 'bar'), ('c', False)],
+        None
+    ]
+    arr = pa.array(data, type=ty)
+    assert arr.to_pylist() == [
+        {'a': 5, 'b': 'foo', 'c': True},
+        {'a': 6, 'b': 'bar', 'c': False},
+        None
+    ]
+
+    # test with duplicated field names
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('a', pa.string()),
+        pa.field('b', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('a', 'foo'), ('b', True)],
+        [('a', 6), ('a', 'bar'), ('b', False)],
+    ]
+    arr = pa.array(data, type=ty)
+    with pytest.raises(KeyError):
+        # TODO(kszucs): ARROW-9997
+        arr.to_pylist()
+
+    # test with empty elements
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [],
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 2), ('b', 'baz')],
+        [('a', 1), ('b', 'bar'), ('c', False), ('d', 'julia')],

Review comment:
       Since the *order* matters when constructing from pairs (based on the test below), I would personally expect additional elements would also not be allowed (BTW, what actually happens if the additional element is not in the last position? Maybe add a test for that as well?). 
   While when constructing from dicts also the order does not matter I suppose? So this could be more tolerant
   
   




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/array.pxi
##########
@@ -21,28 +21,28 @@ import warnings
 
 cdef _sequence_to_array(object sequence, object mask, object size,
                         DataType type, CMemoryPool* pool, c_bool from_pandas):
-    cdef int64_t c_size
-    cdef PyConversionOptions options
+    cdef:
+        int64_t c_size
+        PyConversionOptions options
+        shared_ptr[CChunkedArray] chunked
 
     if type is not None:
         options.type = type.sp_type
 
     if size is not None:
         options.size = size
 
-    options.pool = pool
     options.from_pandas = from_pandas
-    options.ignore_timezone = os.environ.get('PYARROW_IGNORE_TIMEZONE', False)

Review comment:
       We still need it for 2.0.0, so reverting. Nice catch :)




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -1347,64 +841,49 @@ Status ConvertToSequenceAndInferSize(PyObject* obj, PyObject** seq, int64_t* siz
   return Status::OK();
 }
 
-Status ConvertPySequence(PyObject* sequence_source, PyObject* mask,
-                         const PyConversionOptions& options,
-                         std::shared_ptr<ChunkedArray>* out) {
+Result<std::shared_ptr<Array>> ConvertPySequence(PyObject* obj, PyObject* mask,
+                                                 const PyConversionOptions& opts) {
   PyAcquireGIL lock;
 
   PyObject* seq;
   OwnedRef tmp_seq_nanny;
+  PyConversionOptions options = opts;  // copy options struct since we modify it below
 
   std::shared_ptr<DataType> real_type;
 
   int64_t size = options.size;
-  RETURN_NOT_OK(ConvertToSequenceAndInferSize(sequence_source, &seq, &size));
+  RETURN_NOT_OK(ConvertToSequenceAndInferSize(obj, &seq, &size));
   tmp_seq_nanny.reset(seq);
 
   // In some cases, type inference may be "loose", like strings. If the user
   // passed pa.string(), then we will error if we encounter any non-UTF8
   // value. If not, then we will allow the result to be a BinaryArray
-  bool strict_conversions = false;
+  auto copied_options = options;
+  options.strict = false;
 
   if (options.type == nullptr) {
     RETURN_NOT_OK(InferArrowType(seq, mask, options.from_pandas, &real_type));
-    if (options.ignore_timezone && real_type->id() == Type::TIMESTAMP) {
-      const auto& ts_type = checked_cast<const TimestampType&>(*real_type);
-      real_type = timestamp(ts_type.unit());
-    }
+    // TODO(kszucs): remove this
+    // if (options.ignore_timezone && real_type->id() == Type::TIMESTAMP) {
+    //   const auto& ts_type = checked_cast<const TimestampType&>(*real_type);
+    //   real_type = timestamp(ts_type.unit());
+    // }

Review comment:
       The spark integration tests, we need to run those.




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -329,985 +302,602 @@ struct ValueConverter<DurationType> {
         default:
           return Status::UnknownError("Invalid time unit");
       }
+    } else if (PyArray_CheckAnyScalarExact(obj)) {
+      // validate that the numpy scalar has np.datetime64 dtype
+      std::shared_ptr<DataType> numpy_type;
+      RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &numpy_type));
+      if (!numpy_type->Equals(*type)) {
+        return Status::NotImplemented("Expected np.timedelta64 but got: ",
+                                      numpy_type->ToString());
+      }
+      return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
     } else {
       RETURN_NOT_OK(internal::CIntFromPython(obj, &value));
     }
     return value;
   }
 
-  static inline Result<int64_t> FromNumpy(PyObject* obj, TimeUnit::type unit) {
-    // validate that the numpy scalar has np.timedelta64 dtype
-    std::shared_ptr<DataType> type;
-    RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &type));
-    if (type->id() != DurationType::type_id) {
-      // TODO(kszucs): the message should highlight the received numpy dtype
-      return Status::Invalid("Expected np.timedelta64 but got: ", type->ToString());
-    }
-    // validate that the time units are matching
-    if (unit != checked_cast<const DurationType&>(*type).unit()) {
-      return Status::NotImplemented(
-          "Cannot convert NumPy np.timedelta64 objects with differing unit");
-    }
-    // convert the numpy value
-    return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_any_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    return std::move(view);
-  }
-};
+  // The binary-like intermediate representation is PyBytesView because it keeps temporary
+  // python objects alive (non-contiguous memoryview) and stores whether the original
+  // object was unicode encoded or not, which is used for unicode -> bytes coersion if
+  // there is a non-unicode object observed.
 
-template <typename Type>
-struct ValueConverter<Type, enable_if_string_like<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    // strict conversion, force output to be unicode / utf8 and validate that
-    // any binary values are utf8
-    bool is_utf8 = false;
-    PyBytesView view;
-
-    RETURN_NOT_OK(view.FromString(obj, &is_utf8));
-    if (!is_utf8) {
-      return internal::InvalidValue(obj, "was not a utf8 string");
-    }
-    return std::move(view);
+  static Result<PyBytesView> Convert(const BaseBinaryType*, const O&, I obj) {
+    return PyBytesView::FromString(obj);
   }
 
-  static inline Result<PyBytesView> FromPython(PyObject* obj, bool* is_utf8) {
-    PyBytesView view;
-
-    // Non-strict conversion; keep track of whether values are unicode or bytes
-    if (PyUnicode_Check(obj)) {
-      *is_utf8 = true;
-      RETURN_NOT_OK(view.FromUnicode(obj));
+  static Result<PyBytesView> Convert(const FixedSizeBinaryType* type, const O&, I obj) {
+    ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj));
+    if (ARROW_PREDICT_TRUE(view.size == type->byte_width())) {
+      return std::move(view);
     } else {
-      // If not unicode or bytes, FromBinary will error
-      *is_utf8 = false;
-      RETURN_NOT_OK(view.FromBinary(obj));
-    }
-    return std::move(view);
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_fixed_size_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj, int32_t byte_width) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    if (ARROW_PREDICT_FALSE(view.size != byte_width)) {
       std::stringstream ss;
-      ss << "expected to be length " << byte_width << " was " << view.size;
+      ss << "expected to be length " << type->byte_width() << " was " << view.size;
       return internal::InvalidValue(obj, ss.str());
-    } else {
-      return std::move(view);
     }
   }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter base and CRTP "middle" subclasses
-
-class SeqConverter;
-
-// Forward-declare converter factory
-Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
-                    bool strict_conversions, bool ignore_timezone,
-                    std::unique_ptr<SeqConverter>* out);
-
-// Marshal Python sequence (list, tuple, etc.) to Arrow array
-class SeqConverter {
- public:
-  virtual ~SeqConverter() = default;
-
-  // Initialize the sequence converter with an ArrayBuilder created
-  // externally. The reason for this interface is that we have
-  // arrow::MakeBuilder which also creates child builders for nested types, so
-  // we have to pass in the child builders to child SeqConverter in the case of
-  // converting Python objects to Arrow nested types
-  virtual Status Init(ArrayBuilder* builder) = 0;
-
-  // Append a single null value to the builder
-  virtual Status AppendNull() = 0;
-
-  // Append a valid value
-  virtual Status AppendValue(PyObject* seq) = 0;
-
-  // Append a single python object handling Null values
-  virtual Status Append(PyObject* seq) = 0;
 
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status Extend(PyObject* seq, int64_t size) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status ExtendMasked(PyObject* seq, PyObject* mask, int64_t size) = 0;
-
-  virtual Status Close() {
-    if (chunks_.size() == 0 || builder_->length() > 0) {
-      std::shared_ptr<Array> last_chunk;
-      RETURN_NOT_OK(builder_->Finish(&last_chunk));
-      chunks_.emplace_back(std::move(last_chunk));
+  template <typename T>
+  static enable_if_string<T, Result<PyBytesView>> Convert(const T*, const O& options,
+                                                          I obj) {
+    if (options.strict) {
+      // Strict conversion, force output to be unicode / utf8 and validate that
+      // any binary values are utf8
+      ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj, true));
+      if (!view.is_utf8) {
+        return internal::InvalidValue(obj, "was not a utf8 string");
+      }
+      return std::move(view);
+    } else {
+      // Non-strict conversion; keep track of whether values are unicode or bytes
+      return PyBytesView::FromString(obj);
     }
-    return Status::OK();
   }
 
-  virtual Status GetResult(std::shared_ptr<ChunkedArray>* out) {
-    // Still some accumulated data in the builder. If there are no chunks, we
-    // always call Finish to deal with the edge case where a size-0 sequence
-    // was converted with a specific output type, like array([], type=t)
-    RETURN_NOT_OK(Close());
-    *out = std::make_shared<ChunkedArray>(this->chunks_, builder_->type());
-    return Status::OK();
+  static Result<bool> Convert(const DataType* type, const O&, I obj) {
+    return Status::NotImplemented("PyValue::Convert is not implemented for type ", type);
   }
-
-  ArrayBuilder* builder() const { return builder_; }
-
-  int num_chunks() const { return static_cast<int>(chunks_.size()); }
-
- protected:
-  ArrayBuilder* builder_;
-  bool unfinished_builder_;
-  std::vector<std::shared_ptr<Array>> chunks_;
 };
 
-template <typename Type, NullCoding null_coding = NullCoding::NONE_ONLY>
-class TypedConverter : public SeqConverter {
- public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  Status Init(ArrayBuilder* builder) override {
-    builder_ = builder;
-    DCHECK_NE(builder_, nullptr);
-    typed_builder_ = checked_cast<BuilderType*>(builder);
-    return Status::OK();
-  }
-
-  // Append a missing item (default implementation)
-  Status AppendNull() override { return this->typed_builder_->AppendNull(); }
-
-  // Append null if the obj is None or pandas null otherwise the valid value
-  Status Append(PyObject* obj) override {
-    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
-  }
-
-  Status Extend(PyObject* obj, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequence(
-        obj, [this](PyObject* item, bool* /* unused */) { return this->Append(item); });
-  }
-
-  Status ExtendMasked(PyObject* obj, PyObject* mask, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequenceMasked(
-        obj, mask, [this](PyObject* item, bool is_masked, bool* /* unused */) {
-          if (is_masked) {
-            return this->AppendNull();
-          } else {
-            // This will also apply the null-checking convention in the event
-            // that the value is not masked
-            return this->Append(item);  // perhaps use AppendValue instead?
-          }
-        });
-  }
+template <typename T>
+Status Extend(T* converter, PyObject* values, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequence(values, [converter](PyObject* item, bool* /* unused */) {
+    return converter->Append(item);
+  });
+}
 
- protected:
-  BuilderType* typed_builder_;
-};
+// Convert and append a sequence of values masked with a numpy array
+template <typename T>
+Status ExtendMasked(T* converter, PyObject* values, PyObject* mask, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequenceMasked(
+      values, mask, [converter](PyObject* item, bool is_masked, bool* /* unused */) {
+        if (is_masked) {
+          return converter->AppendNull();
+        } else {
+          // This will also apply the null-checking convention in the event
+          // that the value is not masked
+          return converter->Append(item);  // perhaps use AppendValue instead?
+        }
+      });
+}
 
-// ----------------------------------------------------------------------
-// Sequence converter for null type
+// The base Converter class is a mixin with predefined behavior and constructors.
+using PyConverter = Converter<PyObject*, PyConversionOptions>;
 
-template <NullCoding null_coding>
-class NullConverter : public TypedConverter<NullType, null_coding> {
- public:
-  Status AppendValue(PyObject* obj) override {
-    return internal::InvalidValue(obj, "converting to null type");
-  }
-};
+template <typename T, typename Enable = void>
+class PyPrimitiveConverter;
 
-// ----------------------------------------------------------------------
-// Sequence converter template for primitive (integer and floating point bool) types
+template <typename T>
+class PyListConverter;
 
-template <typename Type, NullCoding null_coding>
-class PrimitiveConverter : public TypedConverter<Type, null_coding> {
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value, ValueConverter<Type>::FromPython(obj));
-    return this->typed_builder_->Append(value);
-  }
-};
+template <typename U, typename Enable = void>
+class PyDictionaryConverter;
 
-// ----------------------------------------------------------------------
-// Sequence converters for temporal types
+class PyStructConverter;
 
-template <typename Type, NullCoding null_coding>
-class TimeConverter : public TypedConverter<Type, null_coding> {
- public:
-  explicit TimeConverter(TimeUnit::type unit, bool ignore_timezone)
-      : unit_(unit), ignore_timezone_(ignore_timezone) {}
-
-  // TODO(kszucs): support numpy values for date and time converters
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value,
-                          ValueConverter<Type>::FromPython(obj, unit_, ignore_timezone_));
-    return this->typed_builder_->Append(value);
-  }
+template <typename T, typename Enable = void>
+struct PyConverterTrait;
 
- protected:
-  TimeUnit::type unit_;
-  bool ignore_timezone_;
+template <typename T>
+struct PyConverterTrait<
+    T, enable_if_t<!is_nested_type<T>::value && !is_interval_type<T>::value &&
+                   !is_extension_type<T>::value>> {
+  using type = PyPrimitiveConverter<T>;
 };
 
-// TODO(kszucs): move it to the type_traits
 template <typename T>
-struct NumpyType {};
+struct PyConverterTrait<T, enable_if_list_like<T>> {
+  using type = PyListConverter<T>;
+};
 
 template <>
-struct NumpyType<TimestampType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_DATETIME>::isnull(v);
-  }
+struct PyConverterTrait<StructType> {
+  using type = PyStructConverter;
 };
 
 template <>
-struct NumpyType<DurationType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_TIMEDELTA>::isnull(v);
-  }
+struct PyConverterTrait<DictionaryType> {
+  template <typename T>
+  using type = PyDictionaryConverter<T>;
 };
 
-template <typename Type, NullCoding null_coding>
-class TemporalConverter : public TimeConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_null_type<T>::value || is_boolean_type<T>::value ||
+                   is_number_type<T>::value || is_decimal_type<T>::value ||
+                   is_date_type<T>::value || is_time_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using TimeConverter<Type, null_coding>::TimeConverter;
-
-  Status AppendValue(PyObject* obj) override {
-    int64_t value;
-    if (PyArray_CheckAnyScalarExact(obj)) {
-      // convert np.datetime64 / np.timedelta64 depending on Type
-      ARROW_ASSIGN_OR_RAISE(value, ValueConverter<Type>::FromNumpy(obj, this->unit_));
-      if (NumpyType<Type>::isnull(value)) {
-        // checks numpy NaT sentinel after conversion
-        return this->typed_builder_->AppendNull();
-      }
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
     } else {
       ARROW_ASSIGN_OR_RAISE(
-          value,
-          ValueConverter<Type>::FromPython(
-              obj, this->unit_, TimeConverter<Type, null_coding>::ignore_timezone_));
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      return this->primitive_builder_->Append(converted);
     }
-    return this->typed_builder_->Append(value);
   }
 };
 
-// ----------------------------------------------------------------------
-// Sequence converters for Binary, FixedSizeBinary, String
-
-template <typename Type, NullCoding null_coding>
-class BinaryLikeConverter : public TypedConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_timestamp_type<T>::value || is_duration_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  inline Status AutoChunk(Py_ssize_t size) {
-    // did we reach the builder size limit?
-    if (ARROW_PREDICT_FALSE(this->typed_builder_->value_data_length() + size >
-                            BuilderType::memory_limit())) {
-      // builder would be full, so need to add a new chunk
-      std::shared_ptr<Array> chunk;
-      RETURN_NOT_OK(this->typed_builder_->Finish(&chunk));
-      this->chunks_.emplace_back(std::move(chunk));
-    }
-    return Status::OK();
-  }
-
-  Status AppendString(const PyBytesView& view) {
-    // check that the value fits in the datatype
-    if (view.size > BuilderType::memory_limit()) {
-      return Status::Invalid("string too large for datatype");
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      // Numpy NaT sentinels can be checked after the conversion
+      if (PyArray_CheckAnyScalarExact(value) &&
+          PyValue::IsNaT(this->primitive_type_, converted)) {
+        return this->primitive_builder_->AppendNull();
+      } else {
+        return this->primitive_builder_->Append(converted);
+      }
     }
-    DCHECK_GE(view.size, 0);
-
-    // create a new chunk if the value would overflow the builder
-    RETURN_NOT_OK(AutoChunk(view.size));
-
-    // now we can safely append the value to the builder
-    RETURN_NOT_OK(
-        this->typed_builder_->Append(::arrow::util::string_view(view.bytes, view.size)));
-
-    return Status::OK();
   }
-
- protected:
-  // Create a single instance of PyBytesView here to prevent unnecessary object
-  // creation/destruction
-  PyBytesView string_view_;
 };
 
-template <typename Type, NullCoding null_coding>
-class BinaryConverter : public BinaryLikeConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_binary<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto view, ValueConverter<Type>::FromPython(obj));
-    return this->AppendString(view);
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
+    }
   }
 };
 
-template <NullCoding null_coding>
-class FixedSizeBinaryConverter
-    : public BinaryLikeConverter<FixedSizeBinaryType, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_string_like<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  explicit FixedSizeBinaryConverter(int32_t byte_width) : byte_width_(byte_width) {}
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      if (!view.is_utf8) {
+        // observed binary value
+        observed_binary_ = true;
+      }
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
+    }
+  }
 
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(
-        this->string_view_,
-        ValueConverter<FixedSizeBinaryType>::FromPython(obj, byte_width_));
-    return this->AppendString(this->string_view_);
+  Result<std::shared_ptr<Array>> ToArray() override {
+    ARROW_ASSIGN_OR_RAISE(auto array, (PrimitiveConverter<T, PyConverter>::ToArray()));
+    if (observed_binary_) {
+      // if we saw any non-unicode, cast results to BinaryArray
+      auto binary_type = TypeTraits<typename T::PhysicalType>::type_singleton();
+      return array->View(binary_type);
+    } else {
+      return array;
+    }
   }
 
  protected:
-  int32_t byte_width_;
+  bool observed_binary_ = false;
 };
 
-// For String/UTF8, if strict_conversions enabled, we reject any non-UTF8,
-// otherwise we allow but return results as BinaryArray
-template <typename Type, bool Strict, NullCoding null_coding>
-class StringConverter : public BinaryLikeConverter<Type, null_coding> {
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_c_type<U>>
+    : public DictionaryConverter<U, PyConverter> {
  public:
-  StringConverter() : binary_count_(0) {}
-
-  Status AppendValue(PyObject* obj) override {
-    if (Strict) {
-      // raise if the object is not unicode or not an utf-8 encoded bytes
-      ARROW_ASSIGN_OR_RAISE(this->string_view_, ValueConverter<Type>::FromPython(obj));
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
     } else {
-      // keep track of whether values are unicode or bytes; if any bytes are
-      // observe, the result will be bytes
-      bool is_utf8;
-      ARROW_ASSIGN_OR_RAISE(this->string_view_,
-                            ValueConverter<Type>::FromPython(obj, &is_utf8));
-      if (!is_utf8) {
-        ++binary_count_;
-      }
+      ARROW_ASSIGN_OR_RAISE(auto converted,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(converted);
     }
-    return this->AppendString(this->string_view_);
   }
+};
 
-  Status GetResult(std::shared_ptr<ChunkedArray>* out) override {
-    RETURN_NOT_OK(SeqConverter::GetResult(out));
-
-    // If we saw any non-unicode, cast results to BinaryArray
-    if (binary_count_) {
-      // We should have bailed out earlier
-      DCHECK(!Strict);
-      auto binary_type = TypeTraits<typename Type::PhysicalType>::type_singleton();
-      return (*out)->View(binary_type).Value(out);
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_string_view<U>>
+    : public DictionaryConverter<U, PyConverter> {
+ public:
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(auto view,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(util::string_view(view.bytes, view.size));
     }
-    return Status::OK();
   }
-
- protected:
-  int64_t binary_count_;
 };
 
-// ----------------------------------------------------------------------
-// Convert lists (NumPy arrays containing lists or ndarrays as values)
-
 // If the value type does not match the expected NumPy dtype, then fall through
 // to a slower PySequence-based path
-#define LIST_FAST_CASE(TYPE, NUMPY_TYPE, ArrowType)            \
-  case Type::TYPE: {                                           \
-    if (PyArray_DESCR(arr)->type_num != NUMPY_TYPE) {          \
-      return value_converter_->Extend(obj, value_length);      \
-    }                                                          \
-    return AppendNdarrayTypedItem<NUMPY_TYPE, ArrowType>(arr); \
+#define LIST_FAST_CASE(TYPE_ID, TYPE, NUMPY_TYPE)               \
+  case Type::TYPE_ID: {                                         \
+    if (PyArray_DESCR(ndarray)->type_num != NUMPY_TYPE) {       \
+      return Extend(this->value_converter_.get(), value, size); \
+    }                                                           \
+    return AppendNdarrayTyped<TYPE, NUMPY_TYPE>(ndarray);       \
   }
 
 // Use internal::VisitSequence, fast for NPY_OBJECT but slower otherwise
-#define LIST_SLOW_CASE(TYPE)                            \
-  case Type::TYPE: {                                    \
-    return value_converter_->Extend(obj, value_length); \
+#define LIST_SLOW_CASE(TYPE_ID)                               \
+  case Type::TYPE_ID: {                                       \
+    return Extend(this->value_converter_.get(), value, size); \
   }
 
-// Base class for ListConverter and FixedSizeListConverter (to have both work with CRTP)
-template <typename TypeClass, NullCoding null_coding>
-class BaseListConverter : public TypedConverter<TypeClass, null_coding> {
+template <typename T>
+class PyListConverter : public ListConverter<T, PyConverter, PyConverterTrait> {
  public:
-  using BuilderType = typename TypeTraits<TypeClass>::BuilderType;
-
-  explicit BaseListConverter(bool from_pandas, bool strict_conversions,
-                             bool ignore_timezone)
-      : from_pandas_(from_pandas),
-        strict_conversions_(strict_conversions),
-        ignore_timezone_(ignore_timezone) {}
-
-  Status Init(ArrayBuilder* builder) override {
-    this->builder_ = builder;
-    this->typed_builder_ = checked_cast<BuilderType*>(builder);
-
-    this->value_type_ = checked_cast<const TypeClass&>(*builder->type()).value_type();
-    RETURN_NOT_OK(GetConverter(value_type_, from_pandas_, strict_conversions_,
-                               ignore_timezone_, &value_converter_));
-    return this->value_converter_->Init(this->typed_builder_->value_builder());
-  }
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->list_builder_->AppendNull();
+    }
 
-  template <int NUMPY_TYPE, typename Type>
-  Status AppendNdarrayTypedItem(PyArrayObject* arr) {
-    using traits = internal::npy_traits<NUMPY_TYPE>;
-    using T = typename traits::value_type;
-    using ValueBuilderType = typename TypeTraits<Type>::BuilderType;
+    RETURN_NOT_OK(this->list_builder_->Append());
+    if (PyArray_Check(value)) {
+      RETURN_NOT_OK(AppendNdarray(value));
+    } else if (PySequence_Check(value)) {
+      RETURN_NOT_OK(AppendSequence(value));
+    } else {
+      return internal::InvalidType(
+          value, "was not a sequence or recognized null for conversion to list type");
+    }
 
-    const bool null_sentinels_possible =
-        // Always treat Numpy's NaT as null
-        NUMPY_TYPE == NPY_DATETIME || NUMPY_TYPE == NPY_TIMEDELTA ||
-        // Observing pandas's null sentinels
-        (from_pandas_ && traits::supports_nulls);
+    return ValidateBuilder(this->list_type_);
+  }
 
-    auto child_builder = checked_cast<ValueBuilderType*>(value_converter_->builder());
+ protected:
+  Status ValidateOverflow(const MapType*, int64_t size) { return Status::OK(); }

Review comment:
       Added a test case for it but takes 20 minutes to run. 




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API [WIP]

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



##########
File path: cpp/src/arrow/ipc/metadata_internal.cc
##########
@@ -1159,7 +1159,7 @@ Status GetKeyValueMetadata(const KVVector* fb_metadata,
   auto metadata = std::make_shared<KeyValueMetadata>();
 
   metadata->reserve(fb_metadata->size());
-  for (const auto& pair : *fb_metadata) {
+  for (const auto pair : *fb_metadata) {

Review comment:
       My local toolchain always complains about it, going to remove during the cleanup.




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

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



[GitHub] [arrow] bkietz commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,296 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename BaseConverter, template <typename...> class ConverterTrait>
+static Result<std::shared_ptr<BaseConverter>> MakeConverter(
+    std::shared_ptr<DataType> type, typename BaseConverter::OptionsType options,
+    MemoryPool* pool);
+
+template <typename Input, typename Options>
+class Converter {
+ public:
+  using Self = Converter<Input, Options>;
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  Status Construct(std::shared_ptr<DataType> type, OptionsType options,
+                   MemoryPool* pool) {
+    type_ = std::move(type);
+    options_ = std::move(options);
+    return Init(pool);
+  }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>>& children() const { return children_; }
+
+  Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  virtual Status Init(MemoryPool* pool) { return Status::OK(); }
+
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+ protected:
+  Status Init(MemoryPool* pool) override {
+    this->builder_ = std::make_shared<BuilderType>(this->type_, pool);
+    this->primitive_type_ = checked_cast<const T*>(this->type_.get());
+    this->primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter, template <typename...> class ConverterTrait>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+  using ConverterType = typename ConverterTrait<T>::type;
+
+ protected:
+  Status Init(MemoryPool* pool) override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    ARROW_ASSIGN_OR_RAISE(value_converter_,
+                          (MakeConverter<BaseConverter, ConverterTrait>(
+                              list_type_->value_type(), this->options_, pool)));
+    this->builder_ =
+        std::make_shared<BuilderType>(pool, value_converter_->builder(), this->type_);
+    this->children_ = {value_converter_};
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter, template <typename...> class ConverterTrait>
+class StructConverter : public BaseConverter {
+ public:
+  using ConverterType = typename ConverterTrait<StructType>::type;
+
+ protected:
+  Status Init(MemoryPool* pool) override {
+    std::shared_ptr<BaseConverter> child_converter;
+    std::vector<std::shared_ptr<ArrayBuilder>> child_builders;
+
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    for (const auto& field : struct_type_->fields()) {
+      ARROW_ASSIGN_OR_RAISE(child_converter,
+                            (MakeConverter<BaseConverter, ConverterTrait>(
+                                field->type(), this->options_, pool)));
+      child_builders.push_back(child_converter->builder());
+      this->children_.push_back(std::move(child_converter));
+    }
+
+    this->builder_ =
+        std::make_shared<StructBuilder>(this->type_, pool, std::move(child_builders));
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+
+    return Status::OK();
+  }
+
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+ protected:
+  Status Init(MemoryPool* pool) override {
+    std::unique_ptr<ArrayBuilder> builder;
+    ARROW_RETURN_NOT_OK(MakeDictionaryBuilder(pool, this->type_, NULLPTR, &builder));
+    this->builder_ = std::move(builder);
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename BaseConverter, template <typename...> class ConverterTrait>
+struct MakeConverterImpl {
+  template <typename T, typename ConverterType = typename ConverterTrait<T>::type>
+  Status Visit(const T&) {

Review comment:
       MSVC is [failing to SFINAE](https://ci.appveyor.com/project/ApacheSoftwareFoundation/arrow/builds/35323354/job/0vyq23tsq8a7kwnt#L1015) for `T=DictionaryType`. We'll need to give it some help by changing the dictionary case a bit. Might work:
   
   ```c++
   template <>
   struct PyConverterTrait<DictionaryType> {
     template <typename T>
     using dictionary_type = PyDictionaryConverter<T>;
   };
   ```




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/array.pxi
##########
@@ -158,24 +158,44 @@ def array(object obj, type=None, mask=None, size=None, from_pandas=None,
     Notes
     -----
     Localized timestamps will currently be returned as UTC (pandas's native
-    representation).  Timezone-naive data will be implicitly interpreted as
+    representation). Timezone-naive data will be implicitly interpreted as
     UTC.
 
+    Converting to dictionary array will promote to a wider integer type for
+    indices if the number of distinct values cannot be represented, even if
+    the index type was explicitly set. This means that if there are more than
+    127 values the returned dictionary array's index type will be at least
+    pa.int16() even if pa.int8() was passed to the function. Note that an
+    explicit index type will not be demoted even if it is wider than required.
+
     Examples
     --------
     >>> import pandas as pd
     >>> import pyarrow as pa
     >>> pa.array(pd.Series([1, 2]))
-    <pyarrow.array.Int64Array object at 0x7f674e4c0e10>
+    <pyarrow.lib.Int64Array object at 0x7f674e4c0e10>
     [
       1,
       2
     ]
 
+    >>> pa.array(["a", "b", "a"], type=pa.dictionary(pa.int8(), pa.string()))
+    <pyarrow.lib.DictionaryArray object at 0x7feb288d9040>
+    -- dictionary:
+    [
+      "a",
+      "b"
+    ]
+    -- indices:
+    [
+      0,
+      1,
+      0
+    ]
+
     >>> import numpy as np
-    >>> pa.array(pd.Series([1, 2]), np.array([0, 1],
-    ... dtype=bool))
-    <pyarrow.array.Int64Array object at 0x7f9019e11208>
+    >>> pa.array(pd.Series([1, 2]), np.array([0, 1], dtype=bool))

Review comment:
       Updated.

##########
File path: python/pyarrow/tests/test_types.py
##########
@@ -280,6 +284,13 @@ def test_tzinfo_to_string_errors():
             pa.lib.tzinfo_to_string(tz)
 
 
+@h.given(tzst.timezones())
+def test_timezone_roundtrip(tz):

Review comment:
       Updated.




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

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



[GitHub] [arrow] bkietz commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -329,985 +303,649 @@ struct ValueConverter<DurationType> {
         default:
           return Status::UnknownError("Invalid time unit");
       }
+    } else if (PyArray_CheckAnyScalarExact(obj)) {
+      // validate that the numpy scalar has np.datetime64 dtype
+      std::shared_ptr<DataType> numpy_type;
+      RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &numpy_type));
+      if (!numpy_type->Equals(*type)) {
+        return Status::NotImplemented("Expected np.timedelta64 but got: ",
+                                      numpy_type->ToString());
+      }
+      return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
     } else {
       RETURN_NOT_OK(internal::CIntFromPython(obj, &value));
     }
     return value;
   }
 
-  static inline Result<int64_t> FromNumpy(PyObject* obj, TimeUnit::type unit) {
-    // validate that the numpy scalar has np.timedelta64 dtype
-    std::shared_ptr<DataType> type;
-    RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &type));
-    if (type->id() != DurationType::type_id) {
-      // TODO(kszucs): the message should highlight the received numpy dtype
-      return Status::Invalid("Expected np.timedelta64 but got: ", type->ToString());
-    }
-    // validate that the time units are matching
-    if (unit != checked_cast<const DurationType&>(*type).unit()) {
-      return Status::NotImplemented(
-          "Cannot convert NumPy np.timedelta64 objects with differing unit");
-    }
-    // convert the numpy value
-    return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_any_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    return std::move(view);
-  }
-};
+  // The binary-like intermediate representation is PyBytesView because it keeps temporary
+  // python objects alive (non-contiguous memoryview) and stores whether the original
+  // object was unicode encoded or not, which is used for unicode -> bytes coersion if
+  // there is a non-unicode object observed.
 
-template <typename Type>
-struct ValueConverter<Type, enable_if_string_like<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    // strict conversion, force output to be unicode / utf8 and validate that
-    // any binary values are utf8
-    bool is_utf8 = false;
-    PyBytesView view;
-
-    RETURN_NOT_OK(view.FromString(obj, &is_utf8));
-    if (!is_utf8) {
-      return internal::InvalidValue(obj, "was not a utf8 string");
-    }
-    return std::move(view);
+  static Status Convert(const BaseBinaryType*, const O&, I obj, PyBytesView& view) {
+    return view.ParseString(obj);
   }
 
-  static inline Result<PyBytesView> FromPython(PyObject* obj, bool* is_utf8) {
-    PyBytesView view;
-
-    // Non-strict conversion; keep track of whether values are unicode or bytes
-    if (PyUnicode_Check(obj)) {
-      *is_utf8 = true;
-      RETURN_NOT_OK(view.FromUnicode(obj));
-    } else {
-      // If not unicode or bytes, FromBinary will error
-      *is_utf8 = false;
-      RETURN_NOT_OK(view.FromBinary(obj));
-    }
-    return std::move(view);
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_fixed_size_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj, int32_t byte_width) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    if (ARROW_PREDICT_FALSE(view.size != byte_width)) {
+  static Status Convert(const FixedSizeBinaryType* type, const O&, I obj,
+                        PyBytesView& view) {
+    ARROW_RETURN_NOT_OK(view.ParseString(obj));
+    if (view.size != type->byte_width()) {
       std::stringstream ss;
-      ss << "expected to be length " << byte_width << " was " << view.size;
+      ss << "expected to be length " << type->byte_width() << " was " << view.size;
       return internal::InvalidValue(obj, ss.str());
     } else {
-      return std::move(view);
+      return Status::OK();
     }
   }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter base and CRTP "middle" subclasses
-
-class SeqConverter;
-
-// Forward-declare converter factory
-Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
-                    bool strict_conversions, bool ignore_timezone,
-                    std::unique_ptr<SeqConverter>* out);
-
-// Marshal Python sequence (list, tuple, etc.) to Arrow array
-class SeqConverter {
- public:
-  virtual ~SeqConverter() = default;
-
-  // Initialize the sequence converter with an ArrayBuilder created
-  // externally. The reason for this interface is that we have
-  // arrow::MakeBuilder which also creates child builders for nested types, so
-  // we have to pass in the child builders to child SeqConverter in the case of
-  // converting Python objects to Arrow nested types
-  virtual Status Init(ArrayBuilder* builder) = 0;
-
-  // Append a single null value to the builder
-  virtual Status AppendNull() = 0;
 
-  // Append a valid value
-  virtual Status AppendValue(PyObject* seq) = 0;
-
-  // Append a single python object handling Null values
-  virtual Status Append(PyObject* seq) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status Extend(PyObject* seq, int64_t size) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status ExtendMasked(PyObject* seq, PyObject* mask, int64_t size) = 0;
-
-  virtual Status Close() {
-    if (chunks_.size() == 0 || builder_->length() > 0) {
-      std::shared_ptr<Array> last_chunk;
-      RETURN_NOT_OK(builder_->Finish(&last_chunk));
-      chunks_.emplace_back(std::move(last_chunk));
+  template <typename T>
+  static enable_if_string<T, Status> Convert(const T*, const O& options, I obj,
+                                             PyBytesView& view) {
+    if (options.strict) {
+      // Strict conversion, force output to be unicode / utf8 and validate that
+      // any binary values are utf8
+      ARROW_RETURN_NOT_OK(view.ParseString(obj, true));
+      if (!view.is_utf8) {
+        return internal::InvalidValue(obj, "was not a utf8 string");
+      }
+      return Status::OK();
+    } else {
+      // Non-strict conversion; keep track of whether values are unicode or bytes
+      return view.ParseString(obj);
     }
-    return Status::OK();
   }
 
-  virtual Status GetResult(std::shared_ptr<ChunkedArray>* out) {
-    // Still some accumulated data in the builder. If there are no chunks, we
-    // always call Finish to deal with the edge case where a size-0 sequence
-    // was converted with a specific output type, like array([], type=t)
-    RETURN_NOT_OK(Close());
-    *out = std::make_shared<ChunkedArray>(this->chunks_, builder_->type());
-    return Status::OK();
+  static Result<bool> Convert(const DataType* type, const O&, I obj) {
+    return Status::NotImplemented("PyValue::Convert is not implemented for type ", type);
   }
-
-  ArrayBuilder* builder() const { return builder_; }
-
-  int num_chunks() const { return static_cast<int>(chunks_.size()); }
-
- protected:
-  ArrayBuilder* builder_;
-  bool unfinished_builder_;
-  std::vector<std::shared_ptr<Array>> chunks_;
 };
 
-template <typename Type, NullCoding null_coding = NullCoding::NONE_ONLY>
-class TypedConverter : public SeqConverter {
- public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
+template <typename T>
+Status Extend(T* converter, PyObject* values, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequence(values, [converter](PyObject* item, bool* /* unused */) {
+    return converter->Append(item);
+  });
+}
 
-  Status Init(ArrayBuilder* builder) override {
-    builder_ = builder;
-    DCHECK_NE(builder_, nullptr);
-    typed_builder_ = checked_cast<BuilderType*>(builder);
-    return Status::OK();
-  }
+// Convert and append a sequence of values masked with a numpy array
+template <typename T>
+Status ExtendMasked(T* converter, PyObject* values, PyObject* mask, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequenceMasked(
+      values, mask, [converter](PyObject* item, bool is_masked, bool* /* unused */) {
+        if (is_masked) {
+          return converter->AppendNull();
+        } else {
+          // This will also apply the null-checking convention in the event
+          // that the value is not masked
+          return converter->Append(item);  // perhaps use AppendValue instead?
+        }
+      });
+}
 
-  // Append a missing item (default implementation)
-  Status AppendNull() override { return this->typed_builder_->AppendNull(); }
+// The base Converter class is a mixin with predefined behavior and constructors.
+using PyConverter = Converter<PyObject*, PyConversionOptions>;
 
-  // Append null if the obj is None or pandas null otherwise the valid value
-  Status Append(PyObject* obj) override {
-    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
-  }
+template <typename T, typename Enable = void>
+class PyPrimitiveConverter;
 
-  Status Extend(PyObject* obj, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequence(
-        obj, [this](PyObject* item, bool* /* unused */) { return this->Append(item); });
-  }
+template <typename T>
+class PyListConverter;
 
-  Status ExtendMasked(PyObject* obj, PyObject* mask, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequenceMasked(
-        obj, mask, [this](PyObject* item, bool is_masked, bool* /* unused */) {
-          if (is_masked) {
-            return this->AppendNull();
-          } else {
-            // This will also apply the null-checking convention in the event
-            // that the value is not masked
-            return this->Append(item);  // perhaps use AppendValue instead?
-          }
-        });
-  }
+template <typename U, typename Enable = void>
+class PyDictionaryConverter;
 
- protected:
-  BuilderType* typed_builder_;
-};
+class PyStructConverter;
 
-// ----------------------------------------------------------------------
-// Sequence converter for null type
+template <typename T, typename Enable = void>
+struct PyConverterTrait;
 
-template <NullCoding null_coding>
-class NullConverter : public TypedConverter<NullType, null_coding> {
- public:
-  Status AppendValue(PyObject* obj) override {
-    return internal::InvalidValue(obj, "converting to null type");
-  }
+template <typename T>
+struct PyConverterTrait<
+    T, enable_if_t<!is_nested_type<T>::value && !is_interval_type<T>::value &&
+                   !is_extension_type<T>::value>> {
+  using type = PyPrimitiveConverter<T>;
 };
 
-// ----------------------------------------------------------------------
-// Sequence converter template for primitive (integer and floating point bool) types
-
-template <typename Type, NullCoding null_coding>
-class PrimitiveConverter : public TypedConverter<Type, null_coding> {
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value, ValueConverter<Type>::FromPython(obj));
-    return this->typed_builder_->Append(value);
-  }
+template <typename T>
+struct PyConverterTrait<T, enable_if_list_like<T>> {
+  using type = PyListConverter<T>;
 };
 
-// ----------------------------------------------------------------------
-// Sequence converters for temporal types
-
-template <typename Type, NullCoding null_coding>
-class TimeConverter : public TypedConverter<Type, null_coding> {
- public:
-  explicit TimeConverter(TimeUnit::type unit, bool ignore_timezone)
-      : unit_(unit), ignore_timezone_(ignore_timezone) {}
-
-  // TODO(kszucs): support numpy values for date and time converters
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value,
-                          ValueConverter<Type>::FromPython(obj, unit_, ignore_timezone_));
-    return this->typed_builder_->Append(value);
-  }
-
- protected:
-  TimeUnit::type unit_;
-  bool ignore_timezone_;
+template <>
+struct PyConverterTrait<StructType> {
+  using type = PyStructConverter;
 };
 
-// TODO(kszucs): move it to the type_traits
-template <typename T>
-struct NumpyType {};
-
 template <>
-struct NumpyType<TimestampType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_DATETIME>::isnull(v);
-  }
+struct PyConverterTrait<DictionaryType> {
+  template <typename T>
+  using dictionary_type = PyDictionaryConverter<T>;
 };
 
-template <>
-struct NumpyType<DurationType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_TIMEDELTA>::isnull(v);
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_null<T>>
+    : public PrimitiveConverter<T, PyConverter> {
+ public:
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      return this->primitive_builder_->Append(converted);
+    }
   }
 };
 
-template <typename Type, NullCoding null_coding>
-class TemporalConverter : public TimeConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_boolean_type<T>::value || is_number_type<T>::value ||
+                   is_decimal_type<T>::value || is_date_type<T>::value ||
+                   is_time_type<T>::value>> : public PrimitiveConverter<T, PyConverter> {
  public:
-  using TimeConverter<Type, null_coding>::TimeConverter;
-
-  Status AppendValue(PyObject* obj) override {
-    int64_t value;
-    if (PyArray_CheckAnyScalarExact(obj)) {
-      // convert np.datetime64 / np.timedelta64 depending on Type
-      ARROW_ASSIGN_OR_RAISE(value, ValueConverter<Type>::FromNumpy(obj, this->unit_));
-      if (NumpyType<Type>::isnull(value)) {
-        // checks numpy NaT sentinel after conversion
-        return this->typed_builder_->AppendNull();
-      }
+  Status Append(PyObject* value) override {
+    // Since the required space has been already allocated in the Extend functions we can
+    // rely on the Unsafe builder API which improves the performance.
+    if (PyValue::IsNull(this->options_, value)) {
+      this->primitive_builder_->UnsafeAppendNull();
     } else {
       ARROW_ASSIGN_OR_RAISE(
-          value,
-          ValueConverter<Type>::FromPython(
-              obj, this->unit_, TimeConverter<Type, null_coding>::ignore_timezone_));
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      this->primitive_builder_->UnsafeAppend(converted);
     }
-    return this->typed_builder_->Append(value);
+    return Status::OK();
   }
 };
 
-// ----------------------------------------------------------------------
-// Sequence converters for Binary, FixedSizeBinary, String
-
-template <typename Type, NullCoding null_coding>
-class BinaryLikeConverter : public TypedConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_timestamp_type<T>::value || is_duration_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  inline Status AutoChunk(Py_ssize_t size) {
-    // did we reach the builder size limit?
-    if (ARROW_PREDICT_FALSE(this->typed_builder_->value_data_length() + size >
-                            BuilderType::memory_limit())) {
-      // builder would be full, so need to add a new chunk
-      std::shared_ptr<Array> chunk;
-      RETURN_NOT_OK(this->typed_builder_->Finish(&chunk));
-      this->chunks_.emplace_back(std::move(chunk));
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      this->primitive_builder_->UnsafeAppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      // Numpy NaT sentinels can be checked after the conversion
+      if (PyArray_CheckAnyScalarExact(value) &&
+          PyValue::IsNaT(this->primitive_type_, converted)) {
+        this->primitive_builder_->UnsafeAppendNull();
+      } else {
+        this->primitive_builder_->UnsafeAppend(converted);
+      }
     }
     return Status::OK();
   }
+};
 
-  Status AppendString(const PyBytesView& view) {
-    // check that the value fits in the datatype
-    if (view.size > BuilderType::memory_limit()) {
-      return Status::Invalid("string too large for datatype");
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_binary<T>>
+    : public PrimitiveConverter<T, PyConverter> {
+ public:
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      this->primitive_builder_->UnsafeAppendNull();
+    } else {
+      ARROW_RETURN_NOT_OK(
+          PyValue::Convert(this->primitive_type_, this->options_, value, view_));
+      // Since we don't know the varying length input size in advance, we need to
+      // reserve space in the value builder one by one. ReserveData raises CapacityError
+      // if the value would not fit into the array.
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ReserveData(view_.size));
+      this->primitive_builder_->UnsafeAppend(view_.bytes, view_.size);
     }
-    DCHECK_GE(view.size, 0);
-
-    // create a new chunk if the value would overflow the builder
-    RETURN_NOT_OK(AutoChunk(view.size));
-
-    // now we can safely append the value to the builder
-    RETURN_NOT_OK(
-        this->typed_builder_->Append(::arrow::util::string_view(view.bytes, view.size)));
-
     return Status::OK();
   }
 
  protected:
   // Create a single instance of PyBytesView here to prevent unnecessary object
-  // creation/destruction
-  PyBytesView string_view_;
+  // creation/destruction. This significantly improves the conversion performance.
+  PyBytesView view_;
 };
 
-template <typename Type, NullCoding null_coding>
-class BinaryConverter : public BinaryLikeConverter<Type, null_coding> {
- public:
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto view, ValueConverter<Type>::FromPython(obj));
-    return this->AppendString(view);
-  }
-};
-
-template <NullCoding null_coding>
-class FixedSizeBinaryConverter
-    : public BinaryLikeConverter<FixedSizeBinaryType, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_t<std::is_same<T, FixedSizeBinaryType>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  explicit FixedSizeBinaryConverter(int32_t byte_width) : byte_width_(byte_width) {}
-
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(
-        this->string_view_,
-        ValueConverter<FixedSizeBinaryType>::FromPython(obj, byte_width_));
-    return this->AppendString(this->string_view_);
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      this->primitive_builder_->UnsafeAppendNull();
+    } else {
+      ARROW_RETURN_NOT_OK(
+          PyValue::Convert(this->primitive_type_, this->options_, value, view_));
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ReserveData(view_.size));
+      this->primitive_builder_->UnsafeAppend(view_.bytes);
+    }
+    return Status::OK();
   }
 
  protected:
-  int32_t byte_width_;
+  PyBytesView view_;
 };
 
-// For String/UTF8, if strict_conversions enabled, we reject any non-UTF8,
-// otherwise we allow but return results as BinaryArray
-template <typename Type, bool Strict, NullCoding null_coding>
-class StringConverter : public BinaryLikeConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_string_like<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  StringConverter() : binary_count_(0) {}
-
-  Status AppendValue(PyObject* obj) override {
-    if (Strict) {
-      // raise if the object is not unicode or not an utf-8 encoded bytes
-      ARROW_ASSIGN_OR_RAISE(this->string_view_, ValueConverter<Type>::FromPython(obj));
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      this->primitive_builder_->UnsafeAppendNull();
     } else {
-      // keep track of whether values are unicode or bytes; if any bytes are
-      // observe, the result will be bytes
-      bool is_utf8;
-      ARROW_ASSIGN_OR_RAISE(this->string_view_,
-                            ValueConverter<Type>::FromPython(obj, &is_utf8));
-      if (!is_utf8) {
-        ++binary_count_;
+      ARROW_RETURN_NOT_OK(
+          PyValue::Convert(this->primitive_type_, this->options_, value, view_));
+      if (!view_.is_utf8) {
+        // observed binary value
+        observed_binary_ = true;
       }
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ReserveData(view_.size));
+      this->primitive_builder_->UnsafeAppend(view_.bytes, view_.size);
     }
-    return this->AppendString(this->string_view_);
+    return Status::OK();
   }
 
-  Status GetResult(std::shared_ptr<ChunkedArray>* out) override {
-    RETURN_NOT_OK(SeqConverter::GetResult(out));
-
-    // If we saw any non-unicode, cast results to BinaryArray
-    if (binary_count_) {
-      // We should have bailed out earlier
-      DCHECK(!Strict);
-      auto binary_type = TypeTraits<typename Type::PhysicalType>::type_singleton();
-      return (*out)->View(binary_type).Value(out);
+  Result<std::shared_ptr<Array>> ToArray() override {
+    ARROW_ASSIGN_OR_RAISE(auto array, (PrimitiveConverter<T, PyConverter>::ToArray()));
+    if (observed_binary_) {
+      // if we saw any non-unicode, cast results to BinaryArray
+      auto binary_type = TypeTraits<typename T::PhysicalType>::type_singleton();
+      return array->View(binary_type);
+    } else {
+      return array;
     }
-    return Status::OK();
   }
 
  protected:
-  int64_t binary_count_;
+  PyBytesView view_;
+  bool observed_binary_ = false;
 };
 
-// ----------------------------------------------------------------------
-// Convert lists (NumPy arrays containing lists or ndarrays as values)
-
-// If the value type does not match the expected NumPy dtype, then fall through
-// to a slower PySequence-based path
-#define LIST_FAST_CASE(TYPE, NUMPY_TYPE, ArrowType)            \
-  case Type::TYPE: {                                           \
-    if (PyArray_DESCR(arr)->type_num != NUMPY_TYPE) {          \
-      return value_converter_->Extend(obj, value_length);      \
-    }                                                          \
-    return AppendNdarrayTypedItem<NUMPY_TYPE, ArrowType>(arr); \
-  }
-
-// Use internal::VisitSequence, fast for NPY_OBJECT but slower otherwise
-#define LIST_SLOW_CASE(TYPE)                            \
-  case Type::TYPE: {                                    \
-    return value_converter_->Extend(obj, value_length); \
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_c_type<U>>
+    : public DictionaryConverter<U, PyConverter> {
+ public:
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(auto converted,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(converted);
+    }
   }
+};
 
-// Base class for ListConverter and FixedSizeListConverter (to have both work with CRTP)
-template <typename TypeClass, NullCoding null_coding>
-class BaseListConverter : public TypedConverter<TypeClass, null_coding> {
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_string_view<U>>
+    : public DictionaryConverter<U, PyConverter> {
  public:
-  using BuilderType = typename TypeTraits<TypeClass>::BuilderType;
-
-  explicit BaseListConverter(bool from_pandas, bool strict_conversions,
-                             bool ignore_timezone)
-      : from_pandas_(from_pandas),
-        strict_conversions_(strict_conversions),
-        ignore_timezone_(ignore_timezone) {}
-
-  Status Init(ArrayBuilder* builder) override {
-    this->builder_ = builder;
-    this->typed_builder_ = checked_cast<BuilderType*>(builder);
-
-    this->value_type_ = checked_cast<const TypeClass&>(*builder->type()).value_type();
-    RETURN_NOT_OK(GetConverter(value_type_, from_pandas_, strict_conversions_,
-                               ignore_timezone_, &value_converter_));
-    return this->value_converter_->Init(this->typed_builder_->value_builder());
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
+    } else {
+      ARROW_RETURN_NOT_OK(
+          PyValue::Convert(this->value_type_, this->options_, value, view_));
+      return this->value_builder_->Append(view_.bytes, view_.size);

Review comment:
       You're getting a conversion error here, cast to the expected integer type explicitly
   https://ci.appveyor.com/project/ApacheSoftwareFoundation/arrow/builds/35406728/job/8kbxrx77m3x6mfjy#L1069




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API [WIP]

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,248 @@
+// 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 <datetime.h>
+
+#include <algorithm>
+#include <iostream>
+#include <limits>
+#include <map>
+#include <sstream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/decimal.h"
+#include "arrow/util/int_util_internal.h"
+#include "arrow/util/logging.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename I, typename O>
+class ARROW_EXPORT ArrayConverter {
+ public:
+  using InputType = I;
+  using OptionsType = O;
+
+  ArrayConverter(const std::shared_ptr<DataType>& type,
+                 std::shared_ptr<ArrayBuilder> builder, O options)
+      : sp_type_(type), sp_builder_(builder), options_(options) {}
+
+  virtual ~ArrayConverter() = default;
+  std::shared_ptr<ArrayBuilder> builder() { return sp_builder_; }
+  std::shared_ptr<ArrayBuilder> type() { return sp_type_; }
+  O options() { return options_; }
+
+  virtual Status Init() { return Status::OK(); };
+  virtual Status Reserve(int64_t additional_capacity) = 0;
+
+  virtual Status Append(I value) = 0;
+  virtual Status AppendNull() = 0;
+
+  virtual Status Extend(I seq, int64_t size) = 0;
+
+  virtual Result<std::shared_ptr<Array>> Finish() = 0;
+
+  // virtual Result<std::shared_ptr<Array>> ToArray(I value);
+  // virtual Result<std::shared_ptr<ChunkedArray>> ToChunkedArray(I value);
+
+ protected:
+  const std::shared_ptr<DataType> sp_type_;
+  std::shared_ptr<ArrayBuilder> sp_builder_;
+  O options_;
+};
+
+template <typename T, typename AC>
+class ARROW_EXPORT TypedArrayConverter : public AC {

Review comment:
       I plan to have a short guide in docstrings about its usage.




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,
+                            std::shared_ptr<ArrayBuilder> builder,
+                            const std::vector<std::shared_ptr<Self>>& children,

Review comment:
       Updated.




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

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



[GitHub] [arrow] kszucs commented on pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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


   @github-actions crossbow submit test-spark


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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/includes/libarrow.pxd
##########
@@ -969,11 +969,13 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
         vector[shared_ptr[CScalar]] value
         CResult[shared_ptr[CScalar]] field(CFieldRef ref) const
 
-    cdef cppclass CDictionaryScalar" arrow::DictionaryScalar"(CScalar):
-        cppclass CDictionaryValue "arrow::DictionaryScalar::ValueType":
-            shared_ptr[CScalar] index
-            shared_ptr[CArray] dictionary
+    cdef cppclass CDictionaryValue "arrow::DictionaryScalar::ValueType":
+        shared_ptr[CScalar] index
+        shared_ptr[CArray] dictionary
 
+    cdef cppclass CDictionaryScalar" arrow::DictionaryScalar"(CScalar):
+        CDictionaryScalar(CDictionaryValue value, shared_ptr[CDataType],
+                          c_bool is_valid)
         CDictionaryValue value

Review comment:
       Updated.




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

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



[GitHub] [arrow] kszucs commented on pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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


   @github-actions crossbow submit test-spark


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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter>
+struct MakeConverterImpl;
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  template <typename T>
+  using Primitive = PrimitiveConverter<T, Self>;
+  template <typename T>
+  using List = ListConverter<T, Self>;
+  template <typename T>
+  using Dictionary = DictionaryConverter<T, Self>;
+  using Struct = StructConverter<Self>;
+
+  static Result<std::shared_ptr<Self>> Make(std::shared_ptr<DataType> type,
+                                            MemoryPool* pool, OptionsType options) {
+    std::shared_ptr<Self> out;
+    MakeConverterImpl<Self> visitor = {type, pool, options, &out};
+    ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+    ARROW_RETURN_NOT_OK(out->Init());
+    return out;
+  }
+
+  virtual ~Converter() = default;
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>> children() const { return children_; }
+
+  virtual Status Reserve(int64_t additional_capacity) {

Review comment:
       Removed.




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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/tests/test_types.py
##########
@@ -280,6 +284,13 @@ def test_tzinfo_to_string_errors():
             pa.lib.tzinfo_to_string(tz)
 
 
+@h.given(tzst.timezones())
+def test_timezone_roundtrip(tz):

Review comment:
       this is specifically `pytz` timezone roundtrip, can you clarify that in the test name or a comment?

##########
File path: python/pyarrow/array.pxi
##########
@@ -21,28 +21,28 @@ import warnings
 
 cdef _sequence_to_array(object sequence, object mask, object size,
                         DataType type, CMemoryPool* pool, c_bool from_pandas):
-    cdef int64_t c_size
-    cdef PyConversionOptions options
+    cdef:
+        int64_t c_size
+        PyConversionOptions options
+        shared_ptr[CChunkedArray] chunked
 
     if type is not None:
         options.type = type.sp_type
 
     if size is not None:
         options.size = size
 
-    options.pool = pool
     options.from_pandas = from_pandas
-    options.ignore_timezone = os.environ.get('PYARROW_IGNORE_TIMEZONE', False)

Review comment:
       What's the reason this is being removed? It's handled elsewhere now?

##########
File path: python/pyarrow/scalar.pxi
##########
@@ -610,12 +609,10 @@ cdef class StructScalar(Scalar, collections.abc.Mapping):
     def __getitem__(self, key):
         """
         Return the child value for the given field.
-

Review comment:
       this whitespace was removed by accident? (in any case rst requires an empty line here) Same fore a few lines below

##########
File path: python/pyarrow/array.pxi
##########
@@ -158,24 +158,44 @@ def array(object obj, type=None, mask=None, size=None, from_pandas=None,
     Notes
     -----
     Localized timestamps will currently be returned as UTC (pandas's native
-    representation).  Timezone-naive data will be implicitly interpreted as
+    representation). Timezone-naive data will be implicitly interpreted as
     UTC.
 
+    Converting to dictionary array will promote to a wider integer type for
+    indices if the number of distinct values cannot be represented, even if
+    the index type was explicitly set. This means that if there are more than
+    127 values the returned dictionary array's index type will be at least
+    pa.int16() even if pa.int8() was passed to the function. Note that an
+    explicit index type will not be demoted even if it is wider than required.
+
     Examples
     --------
     >>> import pandas as pd
     >>> import pyarrow as pa
     >>> pa.array(pd.Series([1, 2]))
-    <pyarrow.array.Int64Array object at 0x7f674e4c0e10>
+    <pyarrow.lib.Int64Array object at 0x7f674e4c0e10>
     [
       1,
       2
     ]
 
+    >>> pa.array(["a", "b", "a"], type=pa.dictionary(pa.int8(), pa.string()))
+    <pyarrow.lib.DictionaryArray object at 0x7feb288d9040>
+    -- dictionary:
+    [
+      "a",
+      "b"
+    ]
+    -- indices:
+    [
+      0,
+      1,
+      0
+    ]
+
     >>> import numpy as np
-    >>> pa.array(pd.Series([1, 2]), np.array([0, 1],
-    ... dtype=bool))
-    <pyarrow.array.Int64Array object at 0x7f9019e11208>
+    >>> pa.array(pd.Series([1, 2]), np.array([0, 1], dtype=bool))

Review comment:
       ```suggestion
       >>> pa.array(pd.Series([1, 2]), mask=np.array([0, 1], dtype=bool))
   ```
   
   this example actually doesn't work otherwise

##########
File path: python/pyarrow/tests/test_convert_builtin.py
##########
@@ -1513,6 +1519,108 @@ def test_struct_from_tuples():
             pa.array([tup], type=ty)
 
 
+def test_struct_from_list_of_pairs():
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 6), ('b', 'bar'), ('c', False)],
+        None
+    ]
+    arr = pa.array(data, type=ty)
+    assert arr.to_pylist() == [
+        {'a': 5, 'b': 'foo', 'c': True},
+        {'a': 6, 'b': 'bar', 'c': False},
+        None
+    ]
+
+    # test with duplicated field names
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('a', pa.string()),
+        pa.field('b', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('a', 'foo'), ('b', True)],
+        [('a', 6), ('a', 'bar'), ('b', False)],
+    ]
+    arr = pa.array(data, type=ty)
+    with pytest.raises(KeyError):
+        # TODO(kszucs): ARROW-9997
+        arr.to_pylist()
+
+    # test with empty elements
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [],
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 2), ('b', 'baz')],
+        [('a', 1), ('b', 'bar'), ('c', False), ('d', 'julia')],

Review comment:
       It doesn't raise on "d" being present here?




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

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



[GitHub] [arrow] kszucs commented on pull request #8088: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API [WIP]

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


   Just executed the ASV benchmarks on this PR and the master branch with the two sample sizes and the results showed slighty better performance in both cases:
   
   ```
   ==================== ========= =========
           type            PR      master 
   -------------------- --------- ---------
          int32          136±0ms   138±0ms
          uint32         129±0ms   133±0ms
          int64          131±0ms   138±0ms
          uint64         131±0ms   137±0ms
         float32         131±0ms   131±0ms
         float64         130±0ms   132±0ms
           bool          128±0ms   131±0ms
         decimal         232±0ms   227±0ms
          binary         149±0ms   180±0ms
         binary10        141±0ms   151±0ms
          ascii          158±0ms   160±0ms
         unicode         300±0ms   319±0ms
        int64 list       310±0ms   307±0ms
          struct         224±0ms   241±0ms
    struct from tuples   231±0ms   230±0ms
   ==================== ========= =========
   ```
   
   Also compared the `libarrow_python` binary size: 
   master: 2.7M
   this PR: 2.2M
   
   So far there is no performance degradation - if we can trust the ASV benchmarks. 
   
   


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

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



[GitHub] [arrow] bkietz commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/array.pxi
##########
@@ -158,24 +158,44 @@ def array(object obj, type=None, mask=None, size=None, from_pandas=None,
     Notes
     -----
     Localized timestamps will currently be returned as UTC (pandas's native
-    representation).  Timezone-naive data will be implicitly interpreted as
+    representation). Timezone-naive data will be implicitly interpreted as
     UTC.
 
+    Converting to dictionary array will promote to a wider integer type for
+    indices if the number of distinct values cannot be represented, even if
+    the index type was explicitly set. This means that if there are more than
+    127 values the returned dictionary array's index type will be at least
+    pa.int16() even if pa.int8() was passed to the function. Note that an
+    explicit index type will not be demoted even if it is wider than required.
+
     Examples
     --------
     >>> import pandas as pd
     >>> import pyarrow as pa
     >>> pa.array(pd.Series([1, 2]))
-    <pyarrow.array.Int64Array object at 0x7f674e4c0e10>
+    <pyarrow.lib.Int64Array object at 0x7f674e4c0e10>
     [
       1,
       2
     ]
 
+    >>> pa.array(["a", "b", "a"], type=pa.dictionary(pa.int8(), pa.string()))
+    <pyarrow.lib.DictionaryArray object at 0x7feb288d9040>
+    -- dictionary:
+    [
+      "a",
+      "b"
+    ]
+    -- indices:
+    [
+      0,
+      1,
+      0
+    ]
+

Review comment:
       Would it be worthwhile to add an example of index type promotion?
   ```suggestion
       >>> pa.array(range(1024), type=pa.dictionary(pa.int8(), pa.int64()).type.index_type
       DataType(int16)
   ```

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,
+                            std::shared_ptr<ArrayBuilder> builder,
+                            const std::vector<std::shared_ptr<Self>>& children,
+                            OptionsType options) {
+    type_ = std::move(type);
+    builder_ = std::move(builder);
+    children_ = std::move(children);
+    options_ = std::move(options);
+    return Init();
+  }
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>>& children() const { return children_; }
+
+  Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {

Review comment:
       ```suggestion
   template <typename T, typename BaseConverter, typename ConverterTrait>
   class ListConverter : public BaseConverter {
   ```

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,
+                            std::shared_ptr<ArrayBuilder> builder,
+                            const std::vector<std::shared_ptr<Self>>& children,

Review comment:
       ```suggestion
   ```
   Instead, let converters construct their own children in `Init()`. This will not add to boilerplate at the point of extension since child construction logic can be placed in the mixins. The signature for Initialize should be:
   ```c++
   Status Initialize(std::shared_ptr<DataType> type,
                       OptionsType options,
                       MemoryPool* pool)
   ```

##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -329,985 +302,596 @@ struct ValueConverter<DurationType> {
         default:
           return Status::UnknownError("Invalid time unit");
       }
+    } else if (PyArray_CheckAnyScalarExact(obj)) {
+      // validate that the numpy scalar has np.datetime64 dtype
+      std::shared_ptr<DataType> numpy_type;
+      RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &numpy_type));
+      if (!numpy_type->Equals(*type)) {
+        return Status::NotImplemented("Expected np.timedelta64 but got: ",
+                                      numpy_type->ToString());
+      }
+      return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
     } else {
       RETURN_NOT_OK(internal::CIntFromPython(obj, &value));
     }
     return value;
   }
 
-  static inline Result<int64_t> FromNumpy(PyObject* obj, TimeUnit::type unit) {
-    // validate that the numpy scalar has np.timedelta64 dtype
-    std::shared_ptr<DataType> type;
-    RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &type));
-    if (type->id() != DurationType::type_id) {
-      // TODO(kszucs): the message should highlight the received numpy dtype
-      return Status::Invalid("Expected np.timedelta64 but got: ", type->ToString());
-    }
-    // validate that the time units are matching
-    if (unit != checked_cast<const DurationType&>(*type).unit()) {
-      return Status::NotImplemented(
-          "Cannot convert NumPy np.timedelta64 objects with differing unit");
-    }
-    // convert the numpy value
-    return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_any_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    return std::move(view);
-  }
-};
+  // The binary-like intermediate representation is PyBytesView because it keeps temporary
+  // python objects alive (non-contiguous memoryview) and stores whether the original
+  // object was unicode encoded or not, which is used for unicode -> bytes coersion if
+  // there is a non-unicode object observed.
 
-template <typename Type>
-struct ValueConverter<Type, enable_if_string_like<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    // strict conversion, force output to be unicode / utf8 and validate that
-    // any binary values are utf8
-    bool is_utf8 = false;
-    PyBytesView view;
-
-    RETURN_NOT_OK(view.FromString(obj, &is_utf8));
-    if (!is_utf8) {
-      return internal::InvalidValue(obj, "was not a utf8 string");
-    }
-    return std::move(view);
+  static Result<PyBytesView> Convert(const BaseBinaryType*, const O&, I obj) {
+    return PyBytesView::FromString(obj);
   }
 
-  static inline Result<PyBytesView> FromPython(PyObject* obj, bool* is_utf8) {
-    PyBytesView view;
-
-    // Non-strict conversion; keep track of whether values are unicode or bytes
-    if (PyUnicode_Check(obj)) {
-      *is_utf8 = true;
-      RETURN_NOT_OK(view.FromUnicode(obj));
+  static Result<PyBytesView> Convert(const FixedSizeBinaryType* type, const O&, I obj) {
+    ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj));
+    if (ARROW_PREDICT_TRUE(view.size == type->byte_width())) {
+      return std::move(view);
     } else {
-      // If not unicode or bytes, FromBinary will error
-      *is_utf8 = false;
-      RETURN_NOT_OK(view.FromBinary(obj));
-    }
-    return std::move(view);
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_fixed_size_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj, int32_t byte_width) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    if (ARROW_PREDICT_FALSE(view.size != byte_width)) {
       std::stringstream ss;
-      ss << "expected to be length " << byte_width << " was " << view.size;
+      ss << "expected to be length " << type->byte_width() << " was " << view.size;
       return internal::InvalidValue(obj, ss.str());
-    } else {
-      return std::move(view);
     }
   }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter base and CRTP "middle" subclasses
-
-class SeqConverter;
-
-// Forward-declare converter factory
-Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
-                    bool strict_conversions, bool ignore_timezone,
-                    std::unique_ptr<SeqConverter>* out);
-
-// Marshal Python sequence (list, tuple, etc.) to Arrow array
-class SeqConverter {
- public:
-  virtual ~SeqConverter() = default;
-
-  // Initialize the sequence converter with an ArrayBuilder created
-  // externally. The reason for this interface is that we have
-  // arrow::MakeBuilder which also creates child builders for nested types, so
-  // we have to pass in the child builders to child SeqConverter in the case of
-  // converting Python objects to Arrow nested types
-  virtual Status Init(ArrayBuilder* builder) = 0;
 
-  // Append a single null value to the builder
-  virtual Status AppendNull() = 0;
-
-  // Append a valid value
-  virtual Status AppendValue(PyObject* seq) = 0;
-
-  // Append a single python object handling Null values
-  virtual Status Append(PyObject* seq) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status Extend(PyObject* seq, int64_t size) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status ExtendMasked(PyObject* seq, PyObject* mask, int64_t size) = 0;
-
-  virtual Status Close() {
-    if (chunks_.size() == 0 || builder_->length() > 0) {
-      std::shared_ptr<Array> last_chunk;
-      RETURN_NOT_OK(builder_->Finish(&last_chunk));
-      chunks_.emplace_back(std::move(last_chunk));
+  template <typename T>
+  static enable_if_string<T, Result<PyBytesView>> Convert(const T*, const O& options,
+                                                          I obj) {
+    if (options.strict) {
+      // Strict conversion, force output to be unicode / utf8 and validate that
+      // any binary values are utf8
+      ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj, true));
+      if (!view.is_utf8) {
+        return internal::InvalidValue(obj, "was not a utf8 string");
+      }
+      return std::move(view);
+    } else {
+      // Non-strict conversion; keep track of whether values are unicode or bytes
+      return PyBytesView::FromString(obj);
     }
-    return Status::OK();
   }
 
-  virtual Status GetResult(std::shared_ptr<ChunkedArray>* out) {
-    // Still some accumulated data in the builder. If there are no chunks, we
-    // always call Finish to deal with the edge case where a size-0 sequence
-    // was converted with a specific output type, like array([], type=t)
-    RETURN_NOT_OK(Close());
-    *out = std::make_shared<ChunkedArray>(this->chunks_, builder_->type());
-    return Status::OK();
+  static Result<bool> Convert(const DataType* type, const O&, I obj) {
+    return Status::NotImplemented("PyValue::Convert is not implemented for type ", type);
   }
-
-  ArrayBuilder* builder() const { return builder_; }
-
-  int num_chunks() const { return static_cast<int>(chunks_.size()); }
-
- protected:
-  ArrayBuilder* builder_;
-  bool unfinished_builder_;
-  std::vector<std::shared_ptr<Array>> chunks_;
 };
 
-template <typename Type, NullCoding null_coding = NullCoding::NONE_ONLY>
-class TypedConverter : public SeqConverter {
- public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  Status Init(ArrayBuilder* builder) override {
-    builder_ = builder;
-    DCHECK_NE(builder_, nullptr);
-    typed_builder_ = checked_cast<BuilderType*>(builder);
-    return Status::OK();
-  }
-
-  // Append a missing item (default implementation)
-  Status AppendNull() override { return this->typed_builder_->AppendNull(); }
-
-  // Append null if the obj is None or pandas null otherwise the valid value
-  Status Append(PyObject* obj) override {
-    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
-  }
-
-  Status Extend(PyObject* obj, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequence(
-        obj, [this](PyObject* item, bool* /* unused */) { return this->Append(item); });
-  }
+template <typename T>
+Status Extend(T* converter, PyObject* values, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequence(values, [converter](PyObject* item, bool* /* unused */) {
+    return converter->Append(item);
+  });
+}
 
-  Status ExtendMasked(PyObject* obj, PyObject* mask, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequenceMasked(
-        obj, mask, [this](PyObject* item, bool is_masked, bool* /* unused */) {
-          if (is_masked) {
-            return this->AppendNull();
-          } else {
-            // This will also apply the null-checking convention in the event
-            // that the value is not masked
-            return this->Append(item);  // perhaps use AppendValue instead?
-          }
-        });
-  }
+// Convert and append a sequence of values masked with a numpy array
+template <typename T>
+Status ExtendMasked(T* converter, PyObject* values, PyObject* mask, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequenceMasked(
+      values, mask, [converter](PyObject* item, bool is_masked, bool* /* unused */) {
+        if (is_masked) {
+          return converter->AppendNull();
+        } else {
+          // This will also apply the null-checking convention in the event
+          // that the value is not masked
+          return converter->Append(item);  // perhaps use AppendValue instead?
+        }
+      });
+}
 
- protected:
-  BuilderType* typed_builder_;
-};
+// The base Converter class is a mixin with predefined behavior and constructors.
+class PyConverter : public Converter<PyObject*, PyConversionOptions, PyConverter> {};

Review comment:
       Since this class is now empty, we can remove the third template parameter of `Converter` and simply write:
   ```suggestion
   using PyConverter = Converter<PyObject*, PyConversionOptions>;
   ```
   
   (Within the definition of Converter, just substitute `Converter` for `Self`.)

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,
+                            std::shared_ptr<ArrayBuilder> builder,
+                            const std::vector<std::shared_ptr<Self>>& children,
+                            OptionsType options) {
+    type_ = std::move(type);
+    builder_ = std::move(builder);
+    children_ = std::move(children);
+    options_ = std::move(options);
+    return Init();
+  }
+
+  virtual Status Init() { return Status::OK(); }

Review comment:
       Nit: please rename this method `InitializeImpl()` (or similar) and make it `protected`

##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -329,985 +302,596 @@ struct ValueConverter<DurationType> {
         default:
           return Status::UnknownError("Invalid time unit");
       }
+    } else if (PyArray_CheckAnyScalarExact(obj)) {
+      // validate that the numpy scalar has np.datetime64 dtype
+      std::shared_ptr<DataType> numpy_type;
+      RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &numpy_type));
+      if (!numpy_type->Equals(*type)) {
+        return Status::NotImplemented("Expected np.timedelta64 but got: ",
+                                      numpy_type->ToString());
+      }
+      return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
     } else {
       RETURN_NOT_OK(internal::CIntFromPython(obj, &value));
     }
     return value;
   }
 
-  static inline Result<int64_t> FromNumpy(PyObject* obj, TimeUnit::type unit) {
-    // validate that the numpy scalar has np.timedelta64 dtype
-    std::shared_ptr<DataType> type;
-    RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &type));
-    if (type->id() != DurationType::type_id) {
-      // TODO(kszucs): the message should highlight the received numpy dtype
-      return Status::Invalid("Expected np.timedelta64 but got: ", type->ToString());
-    }
-    // validate that the time units are matching
-    if (unit != checked_cast<const DurationType&>(*type).unit()) {
-      return Status::NotImplemented(
-          "Cannot convert NumPy np.timedelta64 objects with differing unit");
-    }
-    // convert the numpy value
-    return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_any_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    return std::move(view);
-  }
-};
+  // The binary-like intermediate representation is PyBytesView because it keeps temporary
+  // python objects alive (non-contiguous memoryview) and stores whether the original
+  // object was unicode encoded or not, which is used for unicode -> bytes coersion if
+  // there is a non-unicode object observed.
 
-template <typename Type>
-struct ValueConverter<Type, enable_if_string_like<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    // strict conversion, force output to be unicode / utf8 and validate that
-    // any binary values are utf8
-    bool is_utf8 = false;
-    PyBytesView view;
-
-    RETURN_NOT_OK(view.FromString(obj, &is_utf8));
-    if (!is_utf8) {
-      return internal::InvalidValue(obj, "was not a utf8 string");
-    }
-    return std::move(view);
+  static Result<PyBytesView> Convert(const BaseBinaryType*, const O&, I obj) {
+    return PyBytesView::FromString(obj);
   }
 
-  static inline Result<PyBytesView> FromPython(PyObject* obj, bool* is_utf8) {
-    PyBytesView view;
-
-    // Non-strict conversion; keep track of whether values are unicode or bytes
-    if (PyUnicode_Check(obj)) {
-      *is_utf8 = true;
-      RETURN_NOT_OK(view.FromUnicode(obj));
+  static Result<PyBytesView> Convert(const FixedSizeBinaryType* type, const O&, I obj) {
+    ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj));
+    if (ARROW_PREDICT_TRUE(view.size == type->byte_width())) {
+      return std::move(view);
     } else {
-      // If not unicode or bytes, FromBinary will error
-      *is_utf8 = false;
-      RETURN_NOT_OK(view.FromBinary(obj));
-    }
-    return std::move(view);
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_fixed_size_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj, int32_t byte_width) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    if (ARROW_PREDICT_FALSE(view.size != byte_width)) {
       std::stringstream ss;
-      ss << "expected to be length " << byte_width << " was " << view.size;
+      ss << "expected to be length " << type->byte_width() << " was " << view.size;
       return internal::InvalidValue(obj, ss.str());
-    } else {
-      return std::move(view);
     }
   }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter base and CRTP "middle" subclasses
-
-class SeqConverter;
-
-// Forward-declare converter factory
-Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
-                    bool strict_conversions, bool ignore_timezone,
-                    std::unique_ptr<SeqConverter>* out);
-
-// Marshal Python sequence (list, tuple, etc.) to Arrow array
-class SeqConverter {
- public:
-  virtual ~SeqConverter() = default;
-
-  // Initialize the sequence converter with an ArrayBuilder created
-  // externally. The reason for this interface is that we have
-  // arrow::MakeBuilder which also creates child builders for nested types, so
-  // we have to pass in the child builders to child SeqConverter in the case of
-  // converting Python objects to Arrow nested types
-  virtual Status Init(ArrayBuilder* builder) = 0;
 
-  // Append a single null value to the builder
-  virtual Status AppendNull() = 0;
-
-  // Append a valid value
-  virtual Status AppendValue(PyObject* seq) = 0;
-
-  // Append a single python object handling Null values
-  virtual Status Append(PyObject* seq) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status Extend(PyObject* seq, int64_t size) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status ExtendMasked(PyObject* seq, PyObject* mask, int64_t size) = 0;
-
-  virtual Status Close() {
-    if (chunks_.size() == 0 || builder_->length() > 0) {
-      std::shared_ptr<Array> last_chunk;
-      RETURN_NOT_OK(builder_->Finish(&last_chunk));
-      chunks_.emplace_back(std::move(last_chunk));
+  template <typename T>
+  static enable_if_string<T, Result<PyBytesView>> Convert(const T*, const O& options,
+                                                          I obj) {
+    if (options.strict) {
+      // Strict conversion, force output to be unicode / utf8 and validate that
+      // any binary values are utf8
+      ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj, true));
+      if (!view.is_utf8) {
+        return internal::InvalidValue(obj, "was not a utf8 string");
+      }
+      return std::move(view);
+    } else {
+      // Non-strict conversion; keep track of whether values are unicode or bytes
+      return PyBytesView::FromString(obj);
     }
-    return Status::OK();
   }
 
-  virtual Status GetResult(std::shared_ptr<ChunkedArray>* out) {
-    // Still some accumulated data in the builder. If there are no chunks, we
-    // always call Finish to deal with the edge case where a size-0 sequence
-    // was converted with a specific output type, like array([], type=t)
-    RETURN_NOT_OK(Close());
-    *out = std::make_shared<ChunkedArray>(this->chunks_, builder_->type());
-    return Status::OK();
+  static Result<bool> Convert(const DataType* type, const O&, I obj) {
+    return Status::NotImplemented("PyValue::Convert is not implemented for type ", type);
   }
-
-  ArrayBuilder* builder() const { return builder_; }
-
-  int num_chunks() const { return static_cast<int>(chunks_.size()); }
-
- protected:
-  ArrayBuilder* builder_;
-  bool unfinished_builder_;
-  std::vector<std::shared_ptr<Array>> chunks_;
 };
 
-template <typename Type, NullCoding null_coding = NullCoding::NONE_ONLY>
-class TypedConverter : public SeqConverter {
- public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  Status Init(ArrayBuilder* builder) override {
-    builder_ = builder;
-    DCHECK_NE(builder_, nullptr);
-    typed_builder_ = checked_cast<BuilderType*>(builder);
-    return Status::OK();
-  }
-
-  // Append a missing item (default implementation)
-  Status AppendNull() override { return this->typed_builder_->AppendNull(); }
-
-  // Append null if the obj is None or pandas null otherwise the valid value
-  Status Append(PyObject* obj) override {
-    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
-  }
-
-  Status Extend(PyObject* obj, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequence(
-        obj, [this](PyObject* item, bool* /* unused */) { return this->Append(item); });
-  }
+template <typename T>
+Status Extend(T* converter, PyObject* values, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequence(values, [converter](PyObject* item, bool* /* unused */) {
+    return converter->Append(item);
+  });
+}
 
-  Status ExtendMasked(PyObject* obj, PyObject* mask, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequenceMasked(
-        obj, mask, [this](PyObject* item, bool is_masked, bool* /* unused */) {
-          if (is_masked) {
-            return this->AppendNull();
-          } else {
-            // This will also apply the null-checking convention in the event
-            // that the value is not masked
-            return this->Append(item);  // perhaps use AppendValue instead?
-          }
-        });
-  }
+// Convert and append a sequence of values masked with a numpy array
+template <typename T>
+Status ExtendMasked(T* converter, PyObject* values, PyObject* mask, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequenceMasked(
+      values, mask, [converter](PyObject* item, bool is_masked, bool* /* unused */) {
+        if (is_masked) {
+          return converter->AppendNull();
+        } else {
+          // This will also apply the null-checking convention in the event
+          // that the value is not masked
+          return converter->Append(item);  // perhaps use AppendValue instead?
+        }
+      });
+}
 
- protected:
-  BuilderType* typed_builder_;
-};
+// The base Converter class is a mixin with predefined behavior and constructors.
+class PyConverter : public Converter<PyObject*, PyConversionOptions, PyConverter> {};
 
-// ----------------------------------------------------------------------
-// Sequence converter for null type
+template <typename T, typename Enable = void>
+class PyPrimitiveConverter;
 
-template <NullCoding null_coding>
-class NullConverter : public TypedConverter<NullType, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_null_type<T>::value || is_boolean_type<T>::value ||
+                   is_number_type<T>::value || is_decimal_type<T>::value ||
+                   is_date_type<T>::value || is_time_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  Status AppendValue(PyObject* obj) override {
-    return internal::InvalidValue(obj, "converting to null type");
-  }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter template for primitive (integer and floating point bool) types
-
-template <typename Type, NullCoding null_coding>
-class PrimitiveConverter : public TypedConverter<Type, null_coding> {
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value, ValueConverter<Type>::FromPython(obj));
-    return this->typed_builder_->Append(value);
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      return this->primitive_builder_->Append(converted);
+    }
   }
 };
 
-// ----------------------------------------------------------------------
-// Sequence converters for temporal types
-
-template <typename Type, NullCoding null_coding>
-class TimeConverter : public TypedConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_timestamp_type<T>::value || is_duration_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  explicit TimeConverter(TimeUnit::type unit, bool ignore_timezone)
-      : unit_(unit), ignore_timezone_(ignore_timezone) {}
-
-  // TODO(kszucs): support numpy values for date and time converters
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value,
-                          ValueConverter<Type>::FromPython(obj, unit_, ignore_timezone_));
-    return this->typed_builder_->Append(value);
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      // Numpy NaT sentinels can be checked after the conversion
+      if (PyArray_CheckAnyScalarExact(value) &&
+          PyValue::IsNaT(this->primitive_type_, converted)) {
+        return this->primitive_builder_->AppendNull();
+      } else {
+        return this->primitive_builder_->Append(converted);
+      }
+    }
   }
-
- protected:
-  TimeUnit::type unit_;
-  bool ignore_timezone_;
 };
 
-// TODO(kszucs): move it to the type_traits
 template <typename T>
-struct NumpyType {};
-
-template <>
-struct NumpyType<TimestampType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_DATETIME>::isnull(v);
-  }
-};
-
-template <>
-struct NumpyType<DurationType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_TIMEDELTA>::isnull(v);
-  }
-};
-
-template <typename Type, NullCoding null_coding>
-class TemporalConverter : public TimeConverter<Type, null_coding> {
+class PyPrimitiveConverter<T, enable_if_binary<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using TimeConverter<Type, null_coding>::TimeConverter;
-
-  Status AppendValue(PyObject* obj) override {
-    int64_t value;
-    if (PyArray_CheckAnyScalarExact(obj)) {
-      // convert np.datetime64 / np.timedelta64 depending on Type
-      ARROW_ASSIGN_OR_RAISE(value, ValueConverter<Type>::FromNumpy(obj, this->unit_));
-      if (NumpyType<Type>::isnull(value)) {
-        // checks numpy NaT sentinel after conversion
-        return this->typed_builder_->AppendNull();
-      }
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
     } else {
       ARROW_ASSIGN_OR_RAISE(
-          value,
-          ValueConverter<Type>::FromPython(
-              obj, this->unit_, TimeConverter<Type, null_coding>::ignore_timezone_));
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
     }
-    return this->typed_builder_->Append(value);
   }
 };
 
-// ----------------------------------------------------------------------
-// Sequence converters for Binary, FixedSizeBinary, String
-
-template <typename Type, NullCoding null_coding>
-class BinaryLikeConverter : public TypedConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_string_like<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  inline Status AutoChunk(Py_ssize_t size) {
-    // did we reach the builder size limit?
-    if (ARROW_PREDICT_FALSE(this->typed_builder_->value_data_length() + size >
-                            BuilderType::memory_limit())) {
-      // builder would be full, so need to add a new chunk
-      std::shared_ptr<Array> chunk;
-      RETURN_NOT_OK(this->typed_builder_->Finish(&chunk));
-      this->chunks_.emplace_back(std::move(chunk));
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      if (!view.is_utf8) {
+        // observed binary value
+        observed_binary_ = true;
+      }
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
     }
-    return Status::OK();
   }
 
-  Status AppendString(const PyBytesView& view) {
-    // check that the value fits in the datatype
-    if (view.size > BuilderType::memory_limit()) {
-      return Status::Invalid("string too large for datatype");
+  Result<std::shared_ptr<Array>> ToArray() override {
+    ARROW_ASSIGN_OR_RAISE(auto array, (PrimitiveConverter<T, PyConverter>::ToArray()));
+    if (observed_binary_) {
+      // if we saw any non-unicode, cast results to BinaryArray
+      auto binary_type = TypeTraits<typename T::PhysicalType>::type_singleton();
+      return array->View(binary_type);
+    } else {
+      return array;
     }
-    DCHECK_GE(view.size, 0);
-
-    // create a new chunk if the value would overflow the builder
-    RETURN_NOT_OK(AutoChunk(view.size));
-
-    // now we can safely append the value to the builder
-    RETURN_NOT_OK(
-        this->typed_builder_->Append(::arrow::util::string_view(view.bytes, view.size)));
-
-    return Status::OK();
   }
 
  protected:
-  // Create a single instance of PyBytesView here to prevent unnecessary object
-  // creation/destruction
-  PyBytesView string_view_;
+  bool observed_binary_ = false;
 };
 
-template <typename Type, NullCoding null_coding>
-class BinaryConverter : public BinaryLikeConverter<Type, null_coding> {
- public:
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto view, ValueConverter<Type>::FromPython(obj));
-    return this->AppendString(view);
-  }
-};
+template <typename U, typename Enable = void>
+class PyDictionaryConverter;
 
-template <NullCoding null_coding>
-class FixedSizeBinaryConverter
-    : public BinaryLikeConverter<FixedSizeBinaryType, null_coding> {
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_c_type<U>>
+    : public DictionaryConverter<U, PyConverter> {
  public:
-  explicit FixedSizeBinaryConverter(int32_t byte_width) : byte_width_(byte_width) {}
-
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(
-        this->string_view_,
-        ValueConverter<FixedSizeBinaryType>::FromPython(obj, byte_width_));
-    return this->AppendString(this->string_view_);
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(auto converted,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(converted);
+    }
   }
-
- protected:
-  int32_t byte_width_;
 };
 
-// For String/UTF8, if strict_conversions enabled, we reject any non-UTF8,
-// otherwise we allow but return results as BinaryArray
-template <typename Type, bool Strict, NullCoding null_coding>
-class StringConverter : public BinaryLikeConverter<Type, null_coding> {
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_string_view<U>>
+    : public DictionaryConverter<U, PyConverter> {
  public:
-  StringConverter() : binary_count_(0) {}
-
-  Status AppendValue(PyObject* obj) override {
-    if (Strict) {
-      // raise if the object is not unicode or not an utf-8 encoded bytes
-      ARROW_ASSIGN_OR_RAISE(this->string_view_, ValueConverter<Type>::FromPython(obj));
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
     } else {
-      // keep track of whether values are unicode or bytes; if any bytes are
-      // observe, the result will be bytes
-      bool is_utf8;
-      ARROW_ASSIGN_OR_RAISE(this->string_view_,
-                            ValueConverter<Type>::FromPython(obj, &is_utf8));
-      if (!is_utf8) {
-        ++binary_count_;
-      }
+      ARROW_ASSIGN_OR_RAISE(auto view,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(util::string_view(view.bytes, view.size));
     }
-    return this->AppendString(this->string_view_);
   }
-
-  Status GetResult(std::shared_ptr<ChunkedArray>* out) override {
-    RETURN_NOT_OK(SeqConverter::GetResult(out));
-
-    // If we saw any non-unicode, cast results to BinaryArray
-    if (binary_count_) {
-      // We should have bailed out earlier
-      DCHECK(!Strict);
-      auto binary_type = TypeTraits<typename Type::PhysicalType>::type_singleton();
-      return (*out)->View(binary_type).Value(out);
-    }
-    return Status::OK();
-  }
-
- protected:
-  int64_t binary_count_;
 };
 
-// ----------------------------------------------------------------------
-// Convert lists (NumPy arrays containing lists or ndarrays as values)
-
 // If the value type does not match the expected NumPy dtype, then fall through
 // to a slower PySequence-based path
-#define LIST_FAST_CASE(TYPE, NUMPY_TYPE, ArrowType)            \
-  case Type::TYPE: {                                           \
-    if (PyArray_DESCR(arr)->type_num != NUMPY_TYPE) {          \
-      return value_converter_->Extend(obj, value_length);      \
-    }                                                          \
-    return AppendNdarrayTypedItem<NUMPY_TYPE, ArrowType>(arr); \
+#define LIST_FAST_CASE(TYPE_ID, TYPE, NUMPY_TYPE)               \
+  case Type::TYPE_ID: {                                         \
+    if (PyArray_DESCR(ndarray)->type_num != NUMPY_TYPE) {       \
+      return Extend(this->value_converter_.get(), value, size); \
+    }                                                           \
+    return AppendNdarrayTyped<TYPE, NUMPY_TYPE>(ndarray);       \
   }
 
 // Use internal::VisitSequence, fast for NPY_OBJECT but slower otherwise
-#define LIST_SLOW_CASE(TYPE)                            \
-  case Type::TYPE: {                                    \
-    return value_converter_->Extend(obj, value_length); \
+#define LIST_SLOW_CASE(TYPE_ID)                               \
+  case Type::TYPE_ID: {                                       \
+    return Extend(this->value_converter_.get(), value, size); \
   }
 
-// Base class for ListConverter and FixedSizeListConverter (to have both work with CRTP)
-template <typename TypeClass, NullCoding null_coding>
-class BaseListConverter : public TypedConverter<TypeClass, null_coding> {
+template <typename T>
+class PyListConverter : public ListConverter<T, PyConverter> {
  public:
-  using BuilderType = typename TypeTraits<TypeClass>::BuilderType;
-
-  explicit BaseListConverter(bool from_pandas, bool strict_conversions,
-                             bool ignore_timezone)
-      : from_pandas_(from_pandas),
-        strict_conversions_(strict_conversions),
-        ignore_timezone_(ignore_timezone) {}
-
-  Status Init(ArrayBuilder* builder) override {
-    this->builder_ = builder;
-    this->typed_builder_ = checked_cast<BuilderType*>(builder);
-
-    this->value_type_ = checked_cast<const TypeClass&>(*builder->type()).value_type();
-    RETURN_NOT_OK(GetConverter(value_type_, from_pandas_, strict_conversions_,
-                               ignore_timezone_, &value_converter_));
-    return this->value_converter_->Init(this->typed_builder_->value_builder());
+  Status ValidateOverflow(const MapType*, int64_t size) { return Status::OK(); }
+
+  Status ValidateOverflow(const BaseListType*, int64_t size) {
+    return this->list_builder_->ValidateOverflow(size);
   }
 
-  template <int NUMPY_TYPE, typename Type>
-  Status AppendNdarrayTypedItem(PyArrayObject* arr) {
-    using traits = internal::npy_traits<NUMPY_TYPE>;
-    using T = typename traits::value_type;
-    using ValueBuilderType = typename TypeTraits<Type>::BuilderType;
+  Status ValidateBuilder(const MapType*) {
+    if (this->list_builder_->key_builder()->null_count() > 0) {
+      return Status::Invalid("Invalid Map: key field can not contain null values");
+    } else {
+      return Status::OK();
+    }
+  }
 
-    const bool null_sentinels_possible =
-        // Always treat Numpy's NaT as null
-        NUMPY_TYPE == NPY_DATETIME || NUMPY_TYPE == NPY_TIMEDELTA ||
-        // Observing pandas's null sentinels
-        (from_pandas_ && traits::supports_nulls);
+  Status ValidateBuilder(const BaseListType*) { return Status::OK(); }
 
-    auto child_builder = checked_cast<ValueBuilderType*>(value_converter_->builder());
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->list_builder_->AppendNull();
+    }
 
-    // TODO(wesm): Vector append when not strided
-    Ndarray1DIndexer<T> values(arr);
-    if (null_sentinels_possible) {
-      for (int64_t i = 0; i < values.size(); ++i) {
-        if (traits::isnull(values[i])) {
-          RETURN_NOT_OK(child_builder->AppendNull());
-        } else {
-          RETURN_NOT_OK(child_builder->Append(values[i]));
-        }
-      }
+    RETURN_NOT_OK(this->list_builder_->Append());
+    if (PyArray_Check(value)) {
+      RETURN_NOT_OK(AppendNdarray(value));
+    } else if (PySequence_Check(value)) {
+      RETURN_NOT_OK(AppendSequence(value));
     } else {
-      for (int64_t i = 0; i < values.size(); ++i) {
-        RETURN_NOT_OK(child_builder->Append(values[i]));
-      }
+      return internal::InvalidType(
+          value, "was not a sequence or recognized null for conversion to list type");
     }
-    return Status::OK();
+
+    return ValidateBuilder(this->list_type_);
   }
 
-  Status AppendNdarrayItem(PyObject* obj) {
-    PyArrayObject* arr = reinterpret_cast<PyArrayObject*>(obj);
+  Status AppendSequence(PyObject* value) {
+    int64_t size = static_cast<int64_t>(PySequence_Size(value));
+    RETURN_NOT_OK(ValidateOverflow(this->list_type_, size));
+    return Extend(this->value_converter_.get(), value, size);
+  }
 
-    if (PyArray_NDIM(arr) != 1) {
+  Status AppendNdarray(PyObject* value) {
+    PyArrayObject* ndarray = reinterpret_cast<PyArrayObject*>(value);
+    if (PyArray_NDIM(ndarray) != 1) {
       return Status::Invalid("Can only convert 1-dimensional array values");
     }
+    const int64_t size = PyArray_SIZE(ndarray);
+    RETURN_NOT_OK(ValidateOverflow(this->list_type_, size));
 
-    const int64_t value_length = PyArray_SIZE(arr);
-
-    switch (value_type_->id()) {
+    const auto value_type = this->value_converter_->builder()->type();
+    switch (value_type->id()) {
       LIST_SLOW_CASE(NA)
-      LIST_FAST_CASE(UINT8, NPY_UINT8, UInt8Type)
-      LIST_FAST_CASE(INT8, NPY_INT8, Int8Type)
-      LIST_FAST_CASE(UINT16, NPY_UINT16, UInt16Type)
-      LIST_FAST_CASE(INT16, NPY_INT16, Int16Type)
-      LIST_FAST_CASE(UINT32, NPY_UINT32, UInt32Type)
-      LIST_FAST_CASE(INT32, NPY_INT32, Int32Type)
-      LIST_FAST_CASE(UINT64, NPY_UINT64, UInt64Type)
-      LIST_FAST_CASE(INT64, NPY_INT64, Int64Type)
+      LIST_FAST_CASE(UINT8, UInt8Type, NPY_UINT8)
+      LIST_FAST_CASE(INT8, Int8Type, NPY_INT8)
+      LIST_FAST_CASE(UINT16, UInt16Type, NPY_UINT16)
+      LIST_FAST_CASE(INT16, Int16Type, NPY_INT16)
+      LIST_FAST_CASE(UINT32, UInt32Type, NPY_UINT32)
+      LIST_FAST_CASE(INT32, Int32Type, NPY_INT32)
+      LIST_FAST_CASE(UINT64, UInt64Type, NPY_UINT64)
+      LIST_FAST_CASE(INT64, Int64Type, NPY_INT64)
+      LIST_FAST_CASE(HALF_FLOAT, HalfFloatType, NPY_FLOAT16)
+      LIST_FAST_CASE(FLOAT, FloatType, NPY_FLOAT)
+      LIST_FAST_CASE(DOUBLE, DoubleType, NPY_DOUBLE)
+      LIST_FAST_CASE(TIMESTAMP, TimestampType, NPY_DATETIME)
+      LIST_FAST_CASE(DURATION, DurationType, NPY_TIMEDELTA)
       LIST_SLOW_CASE(DATE32)
       LIST_SLOW_CASE(DATE64)
       LIST_SLOW_CASE(TIME32)
       LIST_SLOW_CASE(TIME64)
-      LIST_FAST_CASE(TIMESTAMP, NPY_DATETIME, TimestampType)
-      LIST_FAST_CASE(DURATION, NPY_TIMEDELTA, DurationType)
-      LIST_FAST_CASE(HALF_FLOAT, NPY_FLOAT16, HalfFloatType)
-      LIST_FAST_CASE(FLOAT, NPY_FLOAT, FloatType)
-      LIST_FAST_CASE(DOUBLE, NPY_DOUBLE, DoubleType)
       LIST_SLOW_CASE(BINARY)
       LIST_SLOW_CASE(FIXED_SIZE_BINARY)
       LIST_SLOW_CASE(STRING)
       case Type::LIST: {
-        if (PyArray_DESCR(arr)->type_num != NPY_OBJECT) {
+        if (PyArray_DESCR(ndarray)->type_num != NPY_OBJECT) {
           return Status::Invalid(
-              "Can only convert list types from NumPy object "
-              "array input");
+              "Can only convert list types from NumPy object array input");
         }
-        return internal::VisitSequence(obj, [this](PyObject* item, bool*) {
-          return value_converter_->Append(item);
+        return internal::VisitSequence(value, [this](PyObject* item, bool*) {
+          return this->value_converter_->Append(item);
         });

Review comment:
       ```suggestion
           return Extend(value_converter_.get(), value, /*reserved=*/0);
   ```

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,
+                            std::shared_ptr<ArrayBuilder> builder,
+                            const std::vector<std::shared_ptr<Self>>& children,
+                            OptionsType options) {
+    type_ = std::move(type);
+    builder_ = std::move(builder);
+    children_ = std::move(children);
+    options_ = std::move(options);
+    return Init();
+  }
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>>& children() const { return children_; }
+
+  Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter, template <typename...> class ConverterTrait>
+struct MakeConverterImpl;
+
+template <typename Converter, template <typename...> class ConverterTrait>
+static Result<std::shared_ptr<Converter>> MakeConverter(
+    std::shared_ptr<DataType> type, MemoryPool* pool,
+    typename Converter::OptionsType options) {
+  std::shared_ptr<Converter> out;
+  MakeConverterImpl<Converter, ConverterTrait> visitor = {type, pool, options, &out};
+  ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+  return out;
+}
+
+#define DICTIONARY_CASE(TYPE_ENUM, TYPE_CLASS)                                         \
+  case Type::TYPE_ENUM:                                                                \
+    return Finish<typename ConverterTrait<DictionaryType>::template type<TYPE_CLASS>>( \
+        std::move(builder), {});                                                       \
+    break;
+
+template <typename Converter, template <typename...> class ConverterTrait>
+struct MakeConverterImpl {
+  Status Visit(const NullType& t) {
+    using ConverterType = typename ConverterTrait<NullType>::type;
+
+    auto builder = std::make_shared<NullBuilder>(pool);
+    return Finish<ConverterType>(std::move(builder), {});
+  }
+
+  template <typename T>
+  enable_if_t<!is_nested_type<T>::value && !is_interval_type<T>::value &&
+                  !is_dictionary_type<T>::value && !is_extension_type<T>::value,
+              Status>
+  Visit(const T& t) {
+    using BuilderType = typename TypeTraits<T>::BuilderType;
+    using ConverterType = typename ConverterTrait<T>::type;
+
+    auto builder = std::make_shared<BuilderType>(type, pool);
+    return Finish<ConverterType>(std::move(builder), {});
+  }
+
+  template <typename T>
+  enable_if_t<is_list_like_type<T>::value && !std::is_same<T, MapType>::value, Status>
+  Visit(const T& t) {
+    using BuilderType = typename TypeTraits<T>::BuilderType;
+    using ConverterType = typename ConverterTrait<T>::type;
+
+    ARROW_ASSIGN_OR_RAISE(auto child_converter, (MakeConverter<Converter, ConverterTrait>(
+                                                    t.value_type(), pool, options)));
+    auto builder = std::make_shared<BuilderType>(pool, child_converter->builder(), type);
+    return Finish<ConverterType>(std::move(builder), {std::move(child_converter)});
+  }
+
+  Status Visit(const MapType& t) {
+    using ConverterType = typename ConverterTrait<MapType>::type;
+
+    // TODO(kszucs): seems like builders not respect field nullability
+    std::vector<std::shared_ptr<Field>> struct_fields{t.key_field(), t.item_field()};
+    auto struct_type = std::make_shared<StructType>(struct_fields);
+    ARROW_ASSIGN_OR_RAISE(
+        auto struct_converter,
+        (MakeConverter<Converter, ConverterTrait>(struct_type, pool, options)));
+
+    auto struct_builder = struct_converter->builder();
+    auto key_builder = struct_builder->child_builder(0);
+    auto item_builder = struct_builder->child_builder(1);
+    auto builder = std::make_shared<MapBuilder>(pool, key_builder, item_builder, type);
+
+    return Finish<ConverterType>(std::move(builder), {std::move(struct_converter)});
+  }
+
+  Status Visit(const DictionaryType& t) {
+    std::unique_ptr<ArrayBuilder> builder;
+    ARROW_RETURN_NOT_OK(MakeDictionaryBuilder(pool, type, NULLPTR, &builder));
+
+    switch (t.value_type()->id()) {
+      DICTIONARY_CASE(BOOL, BooleanType);
+      DICTIONARY_CASE(INT8, Int8Type);
+      DICTIONARY_CASE(INT16, Int16Type);
+      DICTIONARY_CASE(INT32, Int32Type);
+      DICTIONARY_CASE(INT64, Int64Type);
+      DICTIONARY_CASE(UINT8, UInt8Type);
+      DICTIONARY_CASE(UINT16, UInt16Type);
+      DICTIONARY_CASE(UINT32, UInt32Type);
+      DICTIONARY_CASE(UINT64, UInt64Type);
+      DICTIONARY_CASE(FLOAT, FloatType);
+      DICTIONARY_CASE(DOUBLE, DoubleType);
+      DICTIONARY_CASE(BINARY, BinaryType);
+      DICTIONARY_CASE(STRING, StringType);
+      DICTIONARY_CASE(FIXED_SIZE_BINARY, FixedSizeBinaryType);
+      default:
+        return Status::NotImplemented("DictionaryArray converter for type ", t.ToString(),
+                                      " not implemented");
+    }
+  }
+
+  Status Visit(const StructType& t) {
+    using ConverterType = typename ConverterTrait<StructType>::type;
+
+    std::shared_ptr<Converter> child_converter;
+    std::vector<std::shared_ptr<Converter>> child_converters;
+    std::vector<std::shared_ptr<ArrayBuilder>> child_builders;
+
+    for (const auto& field : t.fields()) {
+      ARROW_ASSIGN_OR_RAISE(child_converter, (MakeConverter<Converter, ConverterTrait>(
+                                                 field->type(), pool, options)));
+
+      child_builders.push_back(child_converter->builder());
+      child_converters.push_back(std::move(child_converter));
+    }
+
+    auto builder =
+        std::make_shared<StructBuilder>(std::move(type), pool, std::move(child_builders));
+    return Finish<ConverterType>(std::move(builder), std::move(child_converters));
+  }
+
+  Status Visit(const DataType& t) { return Status::NotImplemented(t.name()); }
+
+  template <typename ConverterType>
+  Status Finish(std::shared_ptr<ArrayBuilder> builder,
+                std::vector<std::shared_ptr<Converter>> children) {
+    auto converter = new ConverterType();
+    ARROW_RETURN_NOT_OK(converter->Initialize(std::move(type), std::move(builder),
+                                              std::move(children), std::move(options)));
+    out->reset(converter);
+    return Status::OK();
+  }
+
+  std::shared_ptr<DataType> type;
+  MemoryPool* pool;
+  typename Converter::OptionsType options;
+  std::shared_ptr<Converter>* out;
+};
+
+template <typename Converter>
+class Chunker {
+ public:
+  using Self = Chunker<Converter>;
+  using InputType = typename Converter::InputType;
+
+  explicit Chunker(std::shared_ptr<Converter> converter)
+      : converter_(std::move(converter)) {}
+
+  Status Reserve(int64_t additional_capacity) {
+    return converter_->Reserve(additional_capacity);

Review comment:
       The converter's Reserve() might fail due to overflow, but this shouldn't be an error for Chunker::Reserve since an overflow can be averted by finishing the chunk. See for example the logic surrounding `ChunkedBinaryBuilder::extra_capacity_`. This doesn't need to be handled in this PR since Reserve is a performance hint; for now I think we can just ignore errors emitted by this call to Reserve 

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,

Review comment:
       Since this now defers dynamic dispatch to Init(), it can be made non virtual

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,
+                            std::shared_ptr<ArrayBuilder> builder,
+                            const std::vector<std::shared_ptr<Self>>& children,
+                            OptionsType options) {
+    type_ = std::move(type);
+    builder_ = std::move(builder);
+    children_ = std::move(children);
+    options_ = std::move(options);
+    return Init();
+  }
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>>& children() const { return children_; }
+
+  Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }

Review comment:
       By giving the mixins access to the DataType->ConcreteConverter trait, we can make the construction logic more compact by moving it into `Init()`:
   ```suggestion
     Status Init(MemoryPool* pool) override {
       list_type_ = checked_cast<const T*>(this->type_.get());
       ARROW_ASSIGN_OR_RAISE(value_converter_, (MakeConverter<Converter, ConverterTrait>(
                                                       list_type_->value_type(), pool, this->options_)));
       auto builder = std::make_shared<BuilderType>(pool, value_converter_->builder(), this->type_);
       list_builder_ = builder.get();
       this->builder_ = std::move(builder);
       this->children_ = {value_converter_};
       return Status::OK();
     }
   ```

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,
+                            std::shared_ptr<ArrayBuilder> builder,
+                            const std::vector<std::shared_ptr<Self>>& children,
+                            OptionsType options) {
+    type_ = std::move(type);
+    builder_ = std::move(builder);
+    children_ = std::move(children);
+    options_ = std::move(options);
+    return Init();
+  }
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>>& children() const { return children_; }
+
+  Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter, template <typename...> class ConverterTrait>
+struct MakeConverterImpl;
+
+template <typename Converter, template <typename...> class ConverterTrait>
+static Result<std::shared_ptr<Converter>> MakeConverter(
+    std::shared_ptr<DataType> type, MemoryPool* pool,
+    typename Converter::OptionsType options) {
+  std::shared_ptr<Converter> out;
+  MakeConverterImpl<Converter, ConverterTrait> visitor = {type, pool, options, &out};
+  ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+  return out;
+}

Review comment:
       Once type specific child construction logic has been moved to $Mixins::Init(), we can rewrite MakeConverter as simply:
   ```suggestion
   template <typename Converter, template <typename...> class ConverterTrait>
   struct MakeConverterImpl {
     template <typename T>
     Status Visit(const T&) {
       out = std::make_shared<typename ConverterTrait<T>::type>();
       return out->Initialize(std::move(type), std::move(options), pool);
     }
     
     Status Visit(const DictionaryType& dict_type) {
       switch (dict_type.value_type()->id()) {
   #define DICTIONARY_CASE(TYPE) \
     case TYPE::type_id: \
       out = std::make_shared<typename ConverterTrait<T>::template type<TYPE>>(); \
       break;
         DICTIONARY_CASE(BooleanType);
         DICTIONARY_CASE(Int8Type);
         DICTIONARY_CASE(Int16Type);
         DICTIONARY_CASE(Int32Type);
         DICTIONARY_CASE(Int64Type);
         DICTIONARY_CASE(UInt8Type);
         DICTIONARY_CASE(UInt16Type);
         DICTIONARY_CASE(UInt32Type);
         DICTIONARY_CASE(UInt64Type);
         DICTIONARY_CASE(FloatType);
         DICTIONARY_CASE(DoubleType);
         DICTIONARY_CASE(BinaryType);
         DICTIONARY_CASE(StringType);
         DICTIONARY_CASE(FixedSizeBinaryType);
         default:
           return Status::NotImplemented("DictionaryArray converter for type ", t.ToString(),
                                         " not implemented");
   #undef DICTIONARY_CASE
       }
       return out->Initialize(std::move(type), std::move(options), pool);
     }
     
     std::shared_ptr<DataType> type;
     typename Converter::OptionsType options;
     MemoryPool* pool;
     std::shared_ptr<Converter> out;
   };
   
   template <typename Converter, template <typename...> class ConverterTrait>
   static Result<std::shared_ptr<Converter>> MakeConverter(
       std::shared_ptr<DataType> type, MemoryPool* pool,
       typename Converter::OptionsType options) {
     MakeConverterImpl<Converter, ConverterTrait> visitor{std::move(type), std::move(options), pool, nullptr};
     ARROW_RETURN_NOT_OK(VisitTypeInline(*visitor.type, &visitor));
     return std::move(visitor.out);
   }
   ```




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/scalar.pxi
##########
@@ -639,12 +626,27 @@ cdef class StructScalar(Scalar, collections.abc.Mapping):
             else:
                 raise KeyError(key)
 
+    # TODO(kszucs): consider to remove __iter__, keys and values

Review comment:
       @jorisvandenbossche we need to clean this up once we decide about the desirable API




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,
+                            std::shared_ptr<ArrayBuilder> builder,
+                            const std::vector<std::shared_ptr<Self>>& children,
+                            OptionsType options) {
+    type_ = std::move(type);
+    builder_ = std::move(builder);
+    children_ = std::move(children);
+    options_ = std::move(options);
+    return Init();
+  }
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>>& children() const { return children_; }
+
+  Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }

Review comment:
       Giving access to the mapping trait would require the implementor handle another template parameter. 
   I'd also need to have specific implementations for Null and Map 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.

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -1347,64 +841,49 @@ Status ConvertToSequenceAndInferSize(PyObject* obj, PyObject** seq, int64_t* siz
   return Status::OK();
 }
 
-Status ConvertPySequence(PyObject* sequence_source, PyObject* mask,
-                         const PyConversionOptions& options,
-                         std::shared_ptr<ChunkedArray>* out) {
+Result<std::shared_ptr<Array>> ConvertPySequence(PyObject* obj, PyObject* mask,
+                                                 const PyConversionOptions& opts) {
   PyAcquireGIL lock;
 
   PyObject* seq;
   OwnedRef tmp_seq_nanny;
+  PyConversionOptions options = opts;  // copy options struct since we modify it below
 
   std::shared_ptr<DataType> real_type;
 
   int64_t size = options.size;
-  RETURN_NOT_OK(ConvertToSequenceAndInferSize(sequence_source, &seq, &size));
+  RETURN_NOT_OK(ConvertToSequenceAndInferSize(obj, &seq, &size));
   tmp_seq_nanny.reset(seq);
 
   // In some cases, type inference may be "loose", like strings. If the user
   // passed pa.string(), then we will error if we encounter any non-UTF8
   // value. If not, then we will allow the result to be a BinaryArray
-  bool strict_conversions = false;
+  auto copied_options = options;
+  options.strict = false;
 
   if (options.type == nullptr) {
     RETURN_NOT_OK(InferArrowType(seq, mask, options.from_pandas, &real_type));
-    if (options.ignore_timezone && real_type->id() == Type::TIMESTAMP) {
-      const auto& ts_type = checked_cast<const TimestampType&>(*real_type);
-      real_type = timestamp(ts_type.unit());
-    }
+    // TODO(kszucs): remove this
+    // if (options.ignore_timezone && real_type->id() == Type::TIMESTAMP) {
+    //   const auto& ts_type = checked_cast<const TimestampType&>(*real_type);
+    //   real_type = timestamp(ts_type.unit());
+    // }

Review comment:
       Theoretically yes, but I need to check it hence the todo note.




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/array/array_list_test.cc
##########
@@ -508,6 +534,8 @@ TYPED_TEST(TestListArray, ValidateOffsets) { this->TestValidateOffsets(); }
 
 TYPED_TEST(TestListArray, CornerCases) { this->TestCornerCases(); }
 
+TYPED_TEST(TestListArray, TestOverflowCheck) { this->TestOverflowCheck(); }

Review comment:
       It doesn't seem to work with TYPED_TEST macro, so I expanded it manually.




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

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



[GitHub] [arrow] bkietz commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,296 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename BaseConverter, template <typename...> class ConverterTrait>
+static Result<std::shared_ptr<BaseConverter>> MakeConverter(
+    std::shared_ptr<DataType> type, typename BaseConverter::OptionsType options,
+    MemoryPool* pool);
+
+template <typename Input, typename Options>
+class Converter {
+ public:
+  using Self = Converter<Input, Options>;
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  Status Construct(std::shared_ptr<DataType> type, OptionsType options,
+                   MemoryPool* pool) {
+    type_ = std::move(type);
+    options_ = std::move(options);
+    return Init(pool);
+  }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>>& children() const { return children_; }
+
+  Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  virtual Status Init(MemoryPool* pool) { return Status::OK(); }
+
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+ protected:
+  Status Init(MemoryPool* pool) override {
+    this->builder_ = std::make_shared<BuilderType>(this->type_, pool);
+    this->primitive_type_ = checked_cast<const T*>(this->type_.get());
+    this->primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter, template <typename...> class ConverterTrait>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+  using ConverterType = typename ConverterTrait<T>::type;
+
+ protected:
+  Status Init(MemoryPool* pool) override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    ARROW_ASSIGN_OR_RAISE(value_converter_,
+                          (MakeConverter<BaseConverter, ConverterTrait>(
+                              list_type_->value_type(), this->options_, pool)));
+    this->builder_ =
+        std::make_shared<BuilderType>(pool, value_converter_->builder(), this->type_);
+    this->children_ = {value_converter_};
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter, template <typename...> class ConverterTrait>
+class StructConverter : public BaseConverter {
+ public:
+  using ConverterType = typename ConverterTrait<StructType>::type;
+
+ protected:
+  Status Init(MemoryPool* pool) override {
+    std::shared_ptr<BaseConverter> child_converter;
+    std::vector<std::shared_ptr<ArrayBuilder>> child_builders;
+
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    for (const auto& field : struct_type_->fields()) {
+      ARROW_ASSIGN_OR_RAISE(child_converter,
+                            (MakeConverter<BaseConverter, ConverterTrait>(
+                                field->type(), this->options_, pool)));
+      child_builders.push_back(child_converter->builder());
+      this->children_.push_back(std::move(child_converter));
+    }
+
+    this->builder_ =
+        std::make_shared<StructBuilder>(this->type_, pool, std::move(child_builders));
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+
+    return Status::OK();
+  }
+
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+ protected:
+  Status Init(MemoryPool* pool) override {
+    std::unique_ptr<ArrayBuilder> builder;
+    ARROW_RETURN_NOT_OK(MakeDictionaryBuilder(pool, this->type_, NULLPTR, &builder));
+    this->builder_ = std::move(builder);
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename BaseConverter, template <typename...> class ConverterTrait>
+struct MakeConverterImpl {
+  template <typename T, typename ConverterType = typename ConverterTrait<T>::type>
+  Status Visit(const T&) {
+    out.reset(new ConverterType());
+    return out->Construct(std::move(type), std::move(options), pool);
+  }
+
+  Status Visit(const DictionaryType& t) {
+    switch (t.value_type()->id()) {
+#define DICTIONARY_CASE(TYPE)                                            \
+  case TYPE::type_id:                                                    \
+    out = std::make_shared<                                              \
+        typename ConverterTrait<DictionaryType>::template type<TYPE>>(); \
+    break;
+      DICTIONARY_CASE(BooleanType);
+      DICTIONARY_CASE(Int8Type);
+      DICTIONARY_CASE(Int16Type);
+      DICTIONARY_CASE(Int32Type);
+      DICTIONARY_CASE(Int64Type);
+      DICTIONARY_CASE(UInt8Type);
+      DICTIONARY_CASE(UInt16Type);
+      DICTIONARY_CASE(UInt32Type);
+      DICTIONARY_CASE(UInt64Type);
+      DICTIONARY_CASE(FloatType);
+      DICTIONARY_CASE(DoubleType);
+      DICTIONARY_CASE(BinaryType);
+      DICTIONARY_CASE(StringType);
+      DICTIONARY_CASE(FixedSizeBinaryType);
+      default:
+        return Status::NotImplemented("DictionaryArray converter for type ", t.ToString(),
+                                      " not implemented");
+    }
+    return out->Construct(std::move(type), std::move(options), pool);
+  }
+
+  Status Visit(const DataType& t) { return Status::NotImplemented(t.name()); }
+
+  std::shared_ptr<DataType> type;
+  typename BaseConverter::OptionsType options;
+  MemoryPool* pool;
+  std::shared_ptr<BaseConverter> out;
+};
+
+template <typename BaseConverter, template <typename...> class ConverterTrait>
+static Result<std::shared_ptr<BaseConverter>> MakeConverter(
+    std::shared_ptr<DataType> type, typename BaseConverter::OptionsType options,
+    MemoryPool* pool) {
+  MakeConverterImpl<BaseConverter, ConverterTrait> visitor{
+      std::move(type), std::move(options), pool, nullptr};

Review comment:
       ```suggestion
         std::move(type), std::move(options), pool, NULLPTR};
   ```

##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -329,985 +302,602 @@ struct ValueConverter<DurationType> {
         default:
           return Status::UnknownError("Invalid time unit");
       }
+    } else if (PyArray_CheckAnyScalarExact(obj)) {
+      // validate that the numpy scalar has np.datetime64 dtype
+      std::shared_ptr<DataType> numpy_type;
+      RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &numpy_type));
+      if (!numpy_type->Equals(*type)) {
+        return Status::NotImplemented("Expected np.timedelta64 but got: ",
+                                      numpy_type->ToString());
+      }
+      return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
     } else {
       RETURN_NOT_OK(internal::CIntFromPython(obj, &value));
     }
     return value;
   }
 
-  static inline Result<int64_t> FromNumpy(PyObject* obj, TimeUnit::type unit) {
-    // validate that the numpy scalar has np.timedelta64 dtype
-    std::shared_ptr<DataType> type;
-    RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &type));
-    if (type->id() != DurationType::type_id) {
-      // TODO(kszucs): the message should highlight the received numpy dtype
-      return Status::Invalid("Expected np.timedelta64 but got: ", type->ToString());
-    }
-    // validate that the time units are matching
-    if (unit != checked_cast<const DurationType&>(*type).unit()) {
-      return Status::NotImplemented(
-          "Cannot convert NumPy np.timedelta64 objects with differing unit");
-    }
-    // convert the numpy value
-    return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_any_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    return std::move(view);
-  }
-};
+  // The binary-like intermediate representation is PyBytesView because it keeps temporary
+  // python objects alive (non-contiguous memoryview) and stores whether the original
+  // object was unicode encoded or not, which is used for unicode -> bytes coersion if
+  // there is a non-unicode object observed.
 
-template <typename Type>
-struct ValueConverter<Type, enable_if_string_like<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    // strict conversion, force output to be unicode / utf8 and validate that
-    // any binary values are utf8
-    bool is_utf8 = false;
-    PyBytesView view;
-
-    RETURN_NOT_OK(view.FromString(obj, &is_utf8));
-    if (!is_utf8) {
-      return internal::InvalidValue(obj, "was not a utf8 string");
-    }
-    return std::move(view);
+  static Result<PyBytesView> Convert(const BaseBinaryType*, const O&, I obj) {
+    return PyBytesView::FromString(obj);
   }
 
-  static inline Result<PyBytesView> FromPython(PyObject* obj, bool* is_utf8) {
-    PyBytesView view;
-
-    // Non-strict conversion; keep track of whether values are unicode or bytes
-    if (PyUnicode_Check(obj)) {
-      *is_utf8 = true;
-      RETURN_NOT_OK(view.FromUnicode(obj));
+  static Result<PyBytesView> Convert(const FixedSizeBinaryType* type, const O&, I obj) {
+    ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj));
+    if (ARROW_PREDICT_TRUE(view.size == type->byte_width())) {
+      return std::move(view);
     } else {
-      // If not unicode or bytes, FromBinary will error
-      *is_utf8 = false;
-      RETURN_NOT_OK(view.FromBinary(obj));
-    }
-    return std::move(view);
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_fixed_size_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj, int32_t byte_width) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    if (ARROW_PREDICT_FALSE(view.size != byte_width)) {
       std::stringstream ss;
-      ss << "expected to be length " << byte_width << " was " << view.size;
+      ss << "expected to be length " << type->byte_width() << " was " << view.size;
       return internal::InvalidValue(obj, ss.str());
-    } else {
-      return std::move(view);
     }
   }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter base and CRTP "middle" subclasses
-
-class SeqConverter;
-
-// Forward-declare converter factory
-Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
-                    bool strict_conversions, bool ignore_timezone,
-                    std::unique_ptr<SeqConverter>* out);
-
-// Marshal Python sequence (list, tuple, etc.) to Arrow array
-class SeqConverter {
- public:
-  virtual ~SeqConverter() = default;
-
-  // Initialize the sequence converter with an ArrayBuilder created
-  // externally. The reason for this interface is that we have
-  // arrow::MakeBuilder which also creates child builders for nested types, so
-  // we have to pass in the child builders to child SeqConverter in the case of
-  // converting Python objects to Arrow nested types
-  virtual Status Init(ArrayBuilder* builder) = 0;
-
-  // Append a single null value to the builder
-  virtual Status AppendNull() = 0;
-
-  // Append a valid value
-  virtual Status AppendValue(PyObject* seq) = 0;
-
-  // Append a single python object handling Null values
-  virtual Status Append(PyObject* seq) = 0;
 
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status Extend(PyObject* seq, int64_t size) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status ExtendMasked(PyObject* seq, PyObject* mask, int64_t size) = 0;
-
-  virtual Status Close() {
-    if (chunks_.size() == 0 || builder_->length() > 0) {
-      std::shared_ptr<Array> last_chunk;
-      RETURN_NOT_OK(builder_->Finish(&last_chunk));
-      chunks_.emplace_back(std::move(last_chunk));
+  template <typename T>
+  static enable_if_string<T, Result<PyBytesView>> Convert(const T*, const O& options,
+                                                          I obj) {
+    if (options.strict) {
+      // Strict conversion, force output to be unicode / utf8 and validate that
+      // any binary values are utf8
+      ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj, true));
+      if (!view.is_utf8) {
+        return internal::InvalidValue(obj, "was not a utf8 string");
+      }
+      return std::move(view);
+    } else {
+      // Non-strict conversion; keep track of whether values are unicode or bytes
+      return PyBytesView::FromString(obj);
     }
-    return Status::OK();
   }
 
-  virtual Status GetResult(std::shared_ptr<ChunkedArray>* out) {
-    // Still some accumulated data in the builder. If there are no chunks, we
-    // always call Finish to deal with the edge case where a size-0 sequence
-    // was converted with a specific output type, like array([], type=t)
-    RETURN_NOT_OK(Close());
-    *out = std::make_shared<ChunkedArray>(this->chunks_, builder_->type());
-    return Status::OK();
+  static Result<bool> Convert(const DataType* type, const O&, I obj) {
+    return Status::NotImplemented("PyValue::Convert is not implemented for type ", type);
   }
-
-  ArrayBuilder* builder() const { return builder_; }
-
-  int num_chunks() const { return static_cast<int>(chunks_.size()); }
-
- protected:
-  ArrayBuilder* builder_;
-  bool unfinished_builder_;
-  std::vector<std::shared_ptr<Array>> chunks_;
 };
 
-template <typename Type, NullCoding null_coding = NullCoding::NONE_ONLY>
-class TypedConverter : public SeqConverter {
- public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  Status Init(ArrayBuilder* builder) override {
-    builder_ = builder;
-    DCHECK_NE(builder_, nullptr);
-    typed_builder_ = checked_cast<BuilderType*>(builder);
-    return Status::OK();
-  }
-
-  // Append a missing item (default implementation)
-  Status AppendNull() override { return this->typed_builder_->AppendNull(); }
-
-  // Append null if the obj is None or pandas null otherwise the valid value
-  Status Append(PyObject* obj) override {
-    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
-  }
-
-  Status Extend(PyObject* obj, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequence(
-        obj, [this](PyObject* item, bool* /* unused */) { return this->Append(item); });
-  }
-
-  Status ExtendMasked(PyObject* obj, PyObject* mask, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequenceMasked(
-        obj, mask, [this](PyObject* item, bool is_masked, bool* /* unused */) {
-          if (is_masked) {
-            return this->AppendNull();
-          } else {
-            // This will also apply the null-checking convention in the event
-            // that the value is not masked
-            return this->Append(item);  // perhaps use AppendValue instead?
-          }
-        });
-  }
+template <typename T>
+Status Extend(T* converter, PyObject* values, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequence(values, [converter](PyObject* item, bool* /* unused */) {
+    return converter->Append(item);
+  });
+}
 
- protected:
-  BuilderType* typed_builder_;
-};
+// Convert and append a sequence of values masked with a numpy array
+template <typename T>
+Status ExtendMasked(T* converter, PyObject* values, PyObject* mask, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequenceMasked(
+      values, mask, [converter](PyObject* item, bool is_masked, bool* /* unused */) {
+        if (is_masked) {
+          return converter->AppendNull();
+        } else {
+          // This will also apply the null-checking convention in the event
+          // that the value is not masked
+          return converter->Append(item);  // perhaps use AppendValue instead?
+        }
+      });
+}
 
-// ----------------------------------------------------------------------
-// Sequence converter for null type
+// The base Converter class is a mixin with predefined behavior and constructors.
+using PyConverter = Converter<PyObject*, PyConversionOptions>;
 
-template <NullCoding null_coding>
-class NullConverter : public TypedConverter<NullType, null_coding> {
- public:
-  Status AppendValue(PyObject* obj) override {
-    return internal::InvalidValue(obj, "converting to null type");
-  }
-};
+template <typename T, typename Enable = void>
+class PyPrimitiveConverter;
 
-// ----------------------------------------------------------------------
-// Sequence converter template for primitive (integer and floating point bool) types
+template <typename T>
+class PyListConverter;
 
-template <typename Type, NullCoding null_coding>
-class PrimitiveConverter : public TypedConverter<Type, null_coding> {
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value, ValueConverter<Type>::FromPython(obj));
-    return this->typed_builder_->Append(value);
-  }
-};
+template <typename U, typename Enable = void>
+class PyDictionaryConverter;
 
-// ----------------------------------------------------------------------
-// Sequence converters for temporal types
+class PyStructConverter;
 
-template <typename Type, NullCoding null_coding>
-class TimeConverter : public TypedConverter<Type, null_coding> {
- public:
-  explicit TimeConverter(TimeUnit::type unit, bool ignore_timezone)
-      : unit_(unit), ignore_timezone_(ignore_timezone) {}
-
-  // TODO(kszucs): support numpy values for date and time converters
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value,
-                          ValueConverter<Type>::FromPython(obj, unit_, ignore_timezone_));
-    return this->typed_builder_->Append(value);
-  }
+template <typename T, typename Enable = void>
+struct PyConverterTrait;
 
- protected:
-  TimeUnit::type unit_;
-  bool ignore_timezone_;
+template <typename T>
+struct PyConverterTrait<
+    T, enable_if_t<!is_nested_type<T>::value && !is_interval_type<T>::value &&
+                   !is_extension_type<T>::value>> {
+  using type = PyPrimitiveConverter<T>;
 };
 
-// TODO(kszucs): move it to the type_traits
 template <typename T>
-struct NumpyType {};
+struct PyConverterTrait<T, enable_if_list_like<T>> {
+  using type = PyListConverter<T>;
+};
 
 template <>
-struct NumpyType<TimestampType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_DATETIME>::isnull(v);
-  }
+struct PyConverterTrait<StructType> {
+  using type = PyStructConverter;
 };
 
 template <>
-struct NumpyType<DurationType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_TIMEDELTA>::isnull(v);
-  }
+struct PyConverterTrait<DictionaryType> {
+  template <typename T>
+  using type = PyDictionaryConverter<T>;
 };
 
-template <typename Type, NullCoding null_coding>
-class TemporalConverter : public TimeConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_null_type<T>::value || is_boolean_type<T>::value ||
+                   is_number_type<T>::value || is_decimal_type<T>::value ||
+                   is_date_type<T>::value || is_time_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using TimeConverter<Type, null_coding>::TimeConverter;
-
-  Status AppendValue(PyObject* obj) override {
-    int64_t value;
-    if (PyArray_CheckAnyScalarExact(obj)) {
-      // convert np.datetime64 / np.timedelta64 depending on Type
-      ARROW_ASSIGN_OR_RAISE(value, ValueConverter<Type>::FromNumpy(obj, this->unit_));
-      if (NumpyType<Type>::isnull(value)) {
-        // checks numpy NaT sentinel after conversion
-        return this->typed_builder_->AppendNull();
-      }
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
     } else {
       ARROW_ASSIGN_OR_RAISE(
-          value,
-          ValueConverter<Type>::FromPython(
-              obj, this->unit_, TimeConverter<Type, null_coding>::ignore_timezone_));
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      return this->primitive_builder_->Append(converted);
     }
-    return this->typed_builder_->Append(value);
   }
 };
 
-// ----------------------------------------------------------------------
-// Sequence converters for Binary, FixedSizeBinary, String
-
-template <typename Type, NullCoding null_coding>
-class BinaryLikeConverter : public TypedConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_timestamp_type<T>::value || is_duration_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  inline Status AutoChunk(Py_ssize_t size) {
-    // did we reach the builder size limit?
-    if (ARROW_PREDICT_FALSE(this->typed_builder_->value_data_length() + size >
-                            BuilderType::memory_limit())) {
-      // builder would be full, so need to add a new chunk
-      std::shared_ptr<Array> chunk;
-      RETURN_NOT_OK(this->typed_builder_->Finish(&chunk));
-      this->chunks_.emplace_back(std::move(chunk));
-    }
-    return Status::OK();
-  }
-
-  Status AppendString(const PyBytesView& view) {
-    // check that the value fits in the datatype
-    if (view.size > BuilderType::memory_limit()) {
-      return Status::Invalid("string too large for datatype");
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      // Numpy NaT sentinels can be checked after the conversion
+      if (PyArray_CheckAnyScalarExact(value) &&
+          PyValue::IsNaT(this->primitive_type_, converted)) {
+        return this->primitive_builder_->AppendNull();
+      } else {
+        return this->primitive_builder_->Append(converted);
+      }
     }
-    DCHECK_GE(view.size, 0);
-
-    // create a new chunk if the value would overflow the builder
-    RETURN_NOT_OK(AutoChunk(view.size));
-
-    // now we can safely append the value to the builder
-    RETURN_NOT_OK(
-        this->typed_builder_->Append(::arrow::util::string_view(view.bytes, view.size)));
-
-    return Status::OK();
   }
-
- protected:
-  // Create a single instance of PyBytesView here to prevent unnecessary object
-  // creation/destruction
-  PyBytesView string_view_;
 };
 
-template <typename Type, NullCoding null_coding>
-class BinaryConverter : public BinaryLikeConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_binary<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto view, ValueConverter<Type>::FromPython(obj));
-    return this->AppendString(view);
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
+    }
   }
 };
 
-template <NullCoding null_coding>
-class FixedSizeBinaryConverter
-    : public BinaryLikeConverter<FixedSizeBinaryType, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_string_like<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  explicit FixedSizeBinaryConverter(int32_t byte_width) : byte_width_(byte_width) {}
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      if (!view.is_utf8) {
+        // observed binary value
+        observed_binary_ = true;
+      }
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
+    }
+  }
 
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(
-        this->string_view_,
-        ValueConverter<FixedSizeBinaryType>::FromPython(obj, byte_width_));
-    return this->AppendString(this->string_view_);
+  Result<std::shared_ptr<Array>> ToArray() override {
+    ARROW_ASSIGN_OR_RAISE(auto array, (PrimitiveConverter<T, PyConverter>::ToArray()));
+    if (observed_binary_) {
+      // if we saw any non-unicode, cast results to BinaryArray
+      auto binary_type = TypeTraits<typename T::PhysicalType>::type_singleton();
+      return array->View(binary_type);
+    } else {
+      return array;
+    }
   }
 
  protected:
-  int32_t byte_width_;
+  bool observed_binary_ = false;
 };
 
-// For String/UTF8, if strict_conversions enabled, we reject any non-UTF8,
-// otherwise we allow but return results as BinaryArray
-template <typename Type, bool Strict, NullCoding null_coding>
-class StringConverter : public BinaryLikeConverter<Type, null_coding> {
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_c_type<U>>
+    : public DictionaryConverter<U, PyConverter> {
  public:
-  StringConverter() : binary_count_(0) {}
-
-  Status AppendValue(PyObject* obj) override {
-    if (Strict) {
-      // raise if the object is not unicode or not an utf-8 encoded bytes
-      ARROW_ASSIGN_OR_RAISE(this->string_view_, ValueConverter<Type>::FromPython(obj));
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
     } else {
-      // keep track of whether values are unicode or bytes; if any bytes are
-      // observe, the result will be bytes
-      bool is_utf8;
-      ARROW_ASSIGN_OR_RAISE(this->string_view_,
-                            ValueConverter<Type>::FromPython(obj, &is_utf8));
-      if (!is_utf8) {
-        ++binary_count_;
-      }
+      ARROW_ASSIGN_OR_RAISE(auto converted,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(converted);
     }
-    return this->AppendString(this->string_view_);
   }
+};
 
-  Status GetResult(std::shared_ptr<ChunkedArray>* out) override {
-    RETURN_NOT_OK(SeqConverter::GetResult(out));
-
-    // If we saw any non-unicode, cast results to BinaryArray
-    if (binary_count_) {
-      // We should have bailed out earlier
-      DCHECK(!Strict);
-      auto binary_type = TypeTraits<typename Type::PhysicalType>::type_singleton();
-      return (*out)->View(binary_type).Value(out);
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_string_view<U>>
+    : public DictionaryConverter<U, PyConverter> {
+ public:
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(auto view,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(util::string_view(view.bytes, view.size));
     }
-    return Status::OK();
   }
-
- protected:
-  int64_t binary_count_;
 };
 
-// ----------------------------------------------------------------------
-// Convert lists (NumPy arrays containing lists or ndarrays as values)
-
 // If the value type does not match the expected NumPy dtype, then fall through
 // to a slower PySequence-based path
-#define LIST_FAST_CASE(TYPE, NUMPY_TYPE, ArrowType)            \
-  case Type::TYPE: {                                           \
-    if (PyArray_DESCR(arr)->type_num != NUMPY_TYPE) {          \
-      return value_converter_->Extend(obj, value_length);      \
-    }                                                          \
-    return AppendNdarrayTypedItem<NUMPY_TYPE, ArrowType>(arr); \
+#define LIST_FAST_CASE(TYPE_ID, TYPE, NUMPY_TYPE)               \
+  case Type::TYPE_ID: {                                         \
+    if (PyArray_DESCR(ndarray)->type_num != NUMPY_TYPE) {       \
+      return Extend(this->value_converter_.get(), value, size); \
+    }                                                           \
+    return AppendNdarrayTyped<TYPE, NUMPY_TYPE>(ndarray);       \
   }
 
 // Use internal::VisitSequence, fast for NPY_OBJECT but slower otherwise
-#define LIST_SLOW_CASE(TYPE)                            \
-  case Type::TYPE: {                                    \
-    return value_converter_->Extend(obj, value_length); \
+#define LIST_SLOW_CASE(TYPE_ID)                               \
+  case Type::TYPE_ID: {                                       \
+    return Extend(this->value_converter_.get(), value, size); \
   }
 
-// Base class for ListConverter and FixedSizeListConverter (to have both work with CRTP)
-template <typename TypeClass, NullCoding null_coding>
-class BaseListConverter : public TypedConverter<TypeClass, null_coding> {
+template <typename T>
+class PyListConverter : public ListConverter<T, PyConverter, PyConverterTrait> {
  public:
-  using BuilderType = typename TypeTraits<TypeClass>::BuilderType;
-
-  explicit BaseListConverter(bool from_pandas, bool strict_conversions,
-                             bool ignore_timezone)
-      : from_pandas_(from_pandas),
-        strict_conversions_(strict_conversions),
-        ignore_timezone_(ignore_timezone) {}
-
-  Status Init(ArrayBuilder* builder) override {
-    this->builder_ = builder;
-    this->typed_builder_ = checked_cast<BuilderType*>(builder);
-
-    this->value_type_ = checked_cast<const TypeClass&>(*builder->type()).value_type();
-    RETURN_NOT_OK(GetConverter(value_type_, from_pandas_, strict_conversions_,
-                               ignore_timezone_, &value_converter_));
-    return this->value_converter_->Init(this->typed_builder_->value_builder());
-  }
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->list_builder_->AppendNull();
+    }
 
-  template <int NUMPY_TYPE, typename Type>
-  Status AppendNdarrayTypedItem(PyArrayObject* arr) {
-    using traits = internal::npy_traits<NUMPY_TYPE>;
-    using T = typename traits::value_type;
-    using ValueBuilderType = typename TypeTraits<Type>::BuilderType;
+    RETURN_NOT_OK(this->list_builder_->Append());
+    if (PyArray_Check(value)) {
+      RETURN_NOT_OK(AppendNdarray(value));
+    } else if (PySequence_Check(value)) {
+      RETURN_NOT_OK(AppendSequence(value));
+    } else {
+      return internal::InvalidType(
+          value, "was not a sequence or recognized null for conversion to list type");
+    }
 
-    const bool null_sentinels_possible =
-        // Always treat Numpy's NaT as null
-        NUMPY_TYPE == NPY_DATETIME || NUMPY_TYPE == NPY_TIMEDELTA ||
-        // Observing pandas's null sentinels
-        (from_pandas_ && traits::supports_nulls);
+    return ValidateBuilder(this->list_type_);
+  }
 
-    auto child_builder = checked_cast<ValueBuilderType*>(value_converter_->builder());
+ protected:
+  Status ValidateOverflow(const MapType*, int64_t size) { return Status::OK(); }

Review comment:
       Since MapArray has the same offset structure as ListArray it should probably have non empty overflow validation

##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -329,985 +302,602 @@ struct ValueConverter<DurationType> {
         default:
           return Status::UnknownError("Invalid time unit");
       }
+    } else if (PyArray_CheckAnyScalarExact(obj)) {
+      // validate that the numpy scalar has np.datetime64 dtype
+      std::shared_ptr<DataType> numpy_type;
+      RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &numpy_type));
+      if (!numpy_type->Equals(*type)) {
+        return Status::NotImplemented("Expected np.timedelta64 but got: ",
+                                      numpy_type->ToString());
+      }
+      return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
     } else {
       RETURN_NOT_OK(internal::CIntFromPython(obj, &value));
     }
     return value;
   }
 
-  static inline Result<int64_t> FromNumpy(PyObject* obj, TimeUnit::type unit) {
-    // validate that the numpy scalar has np.timedelta64 dtype
-    std::shared_ptr<DataType> type;
-    RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &type));
-    if (type->id() != DurationType::type_id) {
-      // TODO(kszucs): the message should highlight the received numpy dtype
-      return Status::Invalid("Expected np.timedelta64 but got: ", type->ToString());
-    }
-    // validate that the time units are matching
-    if (unit != checked_cast<const DurationType&>(*type).unit()) {
-      return Status::NotImplemented(
-          "Cannot convert NumPy np.timedelta64 objects with differing unit");
-    }
-    // convert the numpy value
-    return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_any_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    return std::move(view);
-  }
-};
+  // The binary-like intermediate representation is PyBytesView because it keeps temporary
+  // python objects alive (non-contiguous memoryview) and stores whether the original
+  // object was unicode encoded or not, which is used for unicode -> bytes coersion if
+  // there is a non-unicode object observed.
 
-template <typename Type>
-struct ValueConverter<Type, enable_if_string_like<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    // strict conversion, force output to be unicode / utf8 and validate that
-    // any binary values are utf8
-    bool is_utf8 = false;
-    PyBytesView view;
-
-    RETURN_NOT_OK(view.FromString(obj, &is_utf8));
-    if (!is_utf8) {
-      return internal::InvalidValue(obj, "was not a utf8 string");
-    }
-    return std::move(view);
+  static Result<PyBytesView> Convert(const BaseBinaryType*, const O&, I obj) {
+    return PyBytesView::FromString(obj);
   }
 
-  static inline Result<PyBytesView> FromPython(PyObject* obj, bool* is_utf8) {
-    PyBytesView view;
-
-    // Non-strict conversion; keep track of whether values are unicode or bytes
-    if (PyUnicode_Check(obj)) {
-      *is_utf8 = true;
-      RETURN_NOT_OK(view.FromUnicode(obj));
+  static Result<PyBytesView> Convert(const FixedSizeBinaryType* type, const O&, I obj) {
+    ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj));
+    if (ARROW_PREDICT_TRUE(view.size == type->byte_width())) {
+      return std::move(view);
     } else {
-      // If not unicode or bytes, FromBinary will error
-      *is_utf8 = false;
-      RETURN_NOT_OK(view.FromBinary(obj));
-    }
-    return std::move(view);
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_fixed_size_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj, int32_t byte_width) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    if (ARROW_PREDICT_FALSE(view.size != byte_width)) {
       std::stringstream ss;
-      ss << "expected to be length " << byte_width << " was " << view.size;
+      ss << "expected to be length " << type->byte_width() << " was " << view.size;
       return internal::InvalidValue(obj, ss.str());
-    } else {
-      return std::move(view);
     }
   }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter base and CRTP "middle" subclasses
-
-class SeqConverter;
-
-// Forward-declare converter factory
-Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
-                    bool strict_conversions, bool ignore_timezone,
-                    std::unique_ptr<SeqConverter>* out);
-
-// Marshal Python sequence (list, tuple, etc.) to Arrow array
-class SeqConverter {
- public:
-  virtual ~SeqConverter() = default;
-
-  // Initialize the sequence converter with an ArrayBuilder created
-  // externally. The reason for this interface is that we have
-  // arrow::MakeBuilder which also creates child builders for nested types, so
-  // we have to pass in the child builders to child SeqConverter in the case of
-  // converting Python objects to Arrow nested types
-  virtual Status Init(ArrayBuilder* builder) = 0;
-
-  // Append a single null value to the builder
-  virtual Status AppendNull() = 0;
-
-  // Append a valid value
-  virtual Status AppendValue(PyObject* seq) = 0;
-
-  // Append a single python object handling Null values
-  virtual Status Append(PyObject* seq) = 0;
 
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status Extend(PyObject* seq, int64_t size) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status ExtendMasked(PyObject* seq, PyObject* mask, int64_t size) = 0;
-
-  virtual Status Close() {
-    if (chunks_.size() == 0 || builder_->length() > 0) {
-      std::shared_ptr<Array> last_chunk;
-      RETURN_NOT_OK(builder_->Finish(&last_chunk));
-      chunks_.emplace_back(std::move(last_chunk));
+  template <typename T>
+  static enable_if_string<T, Result<PyBytesView>> Convert(const T*, const O& options,
+                                                          I obj) {
+    if (options.strict) {
+      // Strict conversion, force output to be unicode / utf8 and validate that
+      // any binary values are utf8
+      ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj, true));
+      if (!view.is_utf8) {
+        return internal::InvalidValue(obj, "was not a utf8 string");
+      }
+      return std::move(view);
+    } else {
+      // Non-strict conversion; keep track of whether values are unicode or bytes
+      return PyBytesView::FromString(obj);
     }
-    return Status::OK();
   }
 
-  virtual Status GetResult(std::shared_ptr<ChunkedArray>* out) {
-    // Still some accumulated data in the builder. If there are no chunks, we
-    // always call Finish to deal with the edge case where a size-0 sequence
-    // was converted with a specific output type, like array([], type=t)
-    RETURN_NOT_OK(Close());
-    *out = std::make_shared<ChunkedArray>(this->chunks_, builder_->type());
-    return Status::OK();
+  static Result<bool> Convert(const DataType* type, const O&, I obj) {
+    return Status::NotImplemented("PyValue::Convert is not implemented for type ", type);
   }
-
-  ArrayBuilder* builder() const { return builder_; }
-
-  int num_chunks() const { return static_cast<int>(chunks_.size()); }
-
- protected:
-  ArrayBuilder* builder_;
-  bool unfinished_builder_;
-  std::vector<std::shared_ptr<Array>> chunks_;
 };
 
-template <typename Type, NullCoding null_coding = NullCoding::NONE_ONLY>
-class TypedConverter : public SeqConverter {
- public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  Status Init(ArrayBuilder* builder) override {
-    builder_ = builder;
-    DCHECK_NE(builder_, nullptr);
-    typed_builder_ = checked_cast<BuilderType*>(builder);
-    return Status::OK();
-  }
-
-  // Append a missing item (default implementation)
-  Status AppendNull() override { return this->typed_builder_->AppendNull(); }
-
-  // Append null if the obj is None or pandas null otherwise the valid value
-  Status Append(PyObject* obj) override {
-    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
-  }
-
-  Status Extend(PyObject* obj, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequence(
-        obj, [this](PyObject* item, bool* /* unused */) { return this->Append(item); });
-  }
-
-  Status ExtendMasked(PyObject* obj, PyObject* mask, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequenceMasked(
-        obj, mask, [this](PyObject* item, bool is_masked, bool* /* unused */) {
-          if (is_masked) {
-            return this->AppendNull();
-          } else {
-            // This will also apply the null-checking convention in the event
-            // that the value is not masked
-            return this->Append(item);  // perhaps use AppendValue instead?
-          }
-        });
-  }
+template <typename T>
+Status Extend(T* converter, PyObject* values, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequence(values, [converter](PyObject* item, bool* /* unused */) {
+    return converter->Append(item);
+  });
+}
 
- protected:
-  BuilderType* typed_builder_;
-};
+// Convert and append a sequence of values masked with a numpy array
+template <typename T>
+Status ExtendMasked(T* converter, PyObject* values, PyObject* mask, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequenceMasked(
+      values, mask, [converter](PyObject* item, bool is_masked, bool* /* unused */) {
+        if (is_masked) {
+          return converter->AppendNull();
+        } else {
+          // This will also apply the null-checking convention in the event
+          // that the value is not masked
+          return converter->Append(item);  // perhaps use AppendValue instead?
+        }
+      });
+}
 
-// ----------------------------------------------------------------------
-// Sequence converter for null type
+// The base Converter class is a mixin with predefined behavior and constructors.
+using PyConverter = Converter<PyObject*, PyConversionOptions>;
 
-template <NullCoding null_coding>
-class NullConverter : public TypedConverter<NullType, null_coding> {
- public:
-  Status AppendValue(PyObject* obj) override {
-    return internal::InvalidValue(obj, "converting to null type");
-  }
-};
+template <typename T, typename Enable = void>
+class PyPrimitiveConverter;
 
-// ----------------------------------------------------------------------
-// Sequence converter template for primitive (integer and floating point bool) types
+template <typename T>
+class PyListConverter;
 
-template <typename Type, NullCoding null_coding>
-class PrimitiveConverter : public TypedConverter<Type, null_coding> {
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value, ValueConverter<Type>::FromPython(obj));
-    return this->typed_builder_->Append(value);
-  }
-};
+template <typename U, typename Enable = void>
+class PyDictionaryConverter;
 
-// ----------------------------------------------------------------------
-// Sequence converters for temporal types
+class PyStructConverter;
 
-template <typename Type, NullCoding null_coding>
-class TimeConverter : public TypedConverter<Type, null_coding> {
- public:
-  explicit TimeConverter(TimeUnit::type unit, bool ignore_timezone)
-      : unit_(unit), ignore_timezone_(ignore_timezone) {}
-
-  // TODO(kszucs): support numpy values for date and time converters
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value,
-                          ValueConverter<Type>::FromPython(obj, unit_, ignore_timezone_));
-    return this->typed_builder_->Append(value);
-  }
+template <typename T, typename Enable = void>
+struct PyConverterTrait;
 
- protected:
-  TimeUnit::type unit_;
-  bool ignore_timezone_;
+template <typename T>
+struct PyConverterTrait<
+    T, enable_if_t<!is_nested_type<T>::value && !is_interval_type<T>::value &&
+                   !is_extension_type<T>::value>> {
+  using type = PyPrimitiveConverter<T>;
 };
 
-// TODO(kszucs): move it to the type_traits
 template <typename T>
-struct NumpyType {};
+struct PyConverterTrait<T, enable_if_list_like<T>> {
+  using type = PyListConverter<T>;
+};
 
 template <>
-struct NumpyType<TimestampType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_DATETIME>::isnull(v);
-  }
+struct PyConverterTrait<StructType> {
+  using type = PyStructConverter;
 };
 
 template <>
-struct NumpyType<DurationType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_TIMEDELTA>::isnull(v);
-  }
+struct PyConverterTrait<DictionaryType> {
+  template <typename T>
+  using type = PyDictionaryConverter<T>;
 };
 
-template <typename Type, NullCoding null_coding>
-class TemporalConverter : public TimeConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_null_type<T>::value || is_boolean_type<T>::value ||
+                   is_number_type<T>::value || is_decimal_type<T>::value ||
+                   is_date_type<T>::value || is_time_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using TimeConverter<Type, null_coding>::TimeConverter;
-
-  Status AppendValue(PyObject* obj) override {
-    int64_t value;
-    if (PyArray_CheckAnyScalarExact(obj)) {
-      // convert np.datetime64 / np.timedelta64 depending on Type
-      ARROW_ASSIGN_OR_RAISE(value, ValueConverter<Type>::FromNumpy(obj, this->unit_));
-      if (NumpyType<Type>::isnull(value)) {
-        // checks numpy NaT sentinel after conversion
-        return this->typed_builder_->AppendNull();
-      }
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
     } else {
       ARROW_ASSIGN_OR_RAISE(
-          value,
-          ValueConverter<Type>::FromPython(
-              obj, this->unit_, TimeConverter<Type, null_coding>::ignore_timezone_));
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      return this->primitive_builder_->Append(converted);
     }
-    return this->typed_builder_->Append(value);
   }
 };
 
-// ----------------------------------------------------------------------
-// Sequence converters for Binary, FixedSizeBinary, String
-
-template <typename Type, NullCoding null_coding>
-class BinaryLikeConverter : public TypedConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_timestamp_type<T>::value || is_duration_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  inline Status AutoChunk(Py_ssize_t size) {
-    // did we reach the builder size limit?
-    if (ARROW_PREDICT_FALSE(this->typed_builder_->value_data_length() + size >
-                            BuilderType::memory_limit())) {
-      // builder would be full, so need to add a new chunk
-      std::shared_ptr<Array> chunk;
-      RETURN_NOT_OK(this->typed_builder_->Finish(&chunk));
-      this->chunks_.emplace_back(std::move(chunk));
-    }
-    return Status::OK();
-  }
-
-  Status AppendString(const PyBytesView& view) {
-    // check that the value fits in the datatype
-    if (view.size > BuilderType::memory_limit()) {
-      return Status::Invalid("string too large for datatype");
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      // Numpy NaT sentinels can be checked after the conversion
+      if (PyArray_CheckAnyScalarExact(value) &&
+          PyValue::IsNaT(this->primitive_type_, converted)) {
+        return this->primitive_builder_->AppendNull();
+      } else {
+        return this->primitive_builder_->Append(converted);
+      }
     }
-    DCHECK_GE(view.size, 0);
-
-    // create a new chunk if the value would overflow the builder
-    RETURN_NOT_OK(AutoChunk(view.size));
-
-    // now we can safely append the value to the builder
-    RETURN_NOT_OK(
-        this->typed_builder_->Append(::arrow::util::string_view(view.bytes, view.size)));
-
-    return Status::OK();
   }
-
- protected:
-  // Create a single instance of PyBytesView here to prevent unnecessary object
-  // creation/destruction
-  PyBytesView string_view_;
 };
 
-template <typename Type, NullCoding null_coding>
-class BinaryConverter : public BinaryLikeConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_binary<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto view, ValueConverter<Type>::FromPython(obj));
-    return this->AppendString(view);
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
+    }
   }
 };
 
-template <NullCoding null_coding>
-class FixedSizeBinaryConverter
-    : public BinaryLikeConverter<FixedSizeBinaryType, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_string_like<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  explicit FixedSizeBinaryConverter(int32_t byte_width) : byte_width_(byte_width) {}
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      if (!view.is_utf8) {
+        // observed binary value
+        observed_binary_ = true;
+      }
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
+    }
+  }
 
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(
-        this->string_view_,
-        ValueConverter<FixedSizeBinaryType>::FromPython(obj, byte_width_));
-    return this->AppendString(this->string_view_);
+  Result<std::shared_ptr<Array>> ToArray() override {
+    ARROW_ASSIGN_OR_RAISE(auto array, (PrimitiveConverter<T, PyConverter>::ToArray()));
+    if (observed_binary_) {
+      // if we saw any non-unicode, cast results to BinaryArray
+      auto binary_type = TypeTraits<typename T::PhysicalType>::type_singleton();
+      return array->View(binary_type);
+    } else {
+      return array;
+    }
   }
 
  protected:
-  int32_t byte_width_;
+  bool observed_binary_ = false;
 };
 
-// For String/UTF8, if strict_conversions enabled, we reject any non-UTF8,
-// otherwise we allow but return results as BinaryArray
-template <typename Type, bool Strict, NullCoding null_coding>
-class StringConverter : public BinaryLikeConverter<Type, null_coding> {
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_c_type<U>>
+    : public DictionaryConverter<U, PyConverter> {
  public:
-  StringConverter() : binary_count_(0) {}
-
-  Status AppendValue(PyObject* obj) override {
-    if (Strict) {
-      // raise if the object is not unicode or not an utf-8 encoded bytes
-      ARROW_ASSIGN_OR_RAISE(this->string_view_, ValueConverter<Type>::FromPython(obj));
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
     } else {
-      // keep track of whether values are unicode or bytes; if any bytes are
-      // observe, the result will be bytes
-      bool is_utf8;
-      ARROW_ASSIGN_OR_RAISE(this->string_view_,
-                            ValueConverter<Type>::FromPython(obj, &is_utf8));
-      if (!is_utf8) {
-        ++binary_count_;
-      }
+      ARROW_ASSIGN_OR_RAISE(auto converted,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(converted);
     }
-    return this->AppendString(this->string_view_);
   }
+};
 
-  Status GetResult(std::shared_ptr<ChunkedArray>* out) override {
-    RETURN_NOT_OK(SeqConverter::GetResult(out));
-
-    // If we saw any non-unicode, cast results to BinaryArray
-    if (binary_count_) {
-      // We should have bailed out earlier
-      DCHECK(!Strict);
-      auto binary_type = TypeTraits<typename Type::PhysicalType>::type_singleton();
-      return (*out)->View(binary_type).Value(out);
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_string_view<U>>
+    : public DictionaryConverter<U, PyConverter> {
+ public:
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(auto view,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(util::string_view(view.bytes, view.size));
     }
-    return Status::OK();
   }
-
- protected:
-  int64_t binary_count_;
 };
 
-// ----------------------------------------------------------------------
-// Convert lists (NumPy arrays containing lists or ndarrays as values)
-
 // If the value type does not match the expected NumPy dtype, then fall through
 // to a slower PySequence-based path
-#define LIST_FAST_CASE(TYPE, NUMPY_TYPE, ArrowType)            \
-  case Type::TYPE: {                                           \
-    if (PyArray_DESCR(arr)->type_num != NUMPY_TYPE) {          \
-      return value_converter_->Extend(obj, value_length);      \
-    }                                                          \
-    return AppendNdarrayTypedItem<NUMPY_TYPE, ArrowType>(arr); \
+#define LIST_FAST_CASE(TYPE_ID, TYPE, NUMPY_TYPE)               \
+  case Type::TYPE_ID: {                                         \
+    if (PyArray_DESCR(ndarray)->type_num != NUMPY_TYPE) {       \
+      return Extend(this->value_converter_.get(), value, size); \
+    }                                                           \
+    return AppendNdarrayTyped<TYPE, NUMPY_TYPE>(ndarray);       \
   }
 
 // Use internal::VisitSequence, fast for NPY_OBJECT but slower otherwise
-#define LIST_SLOW_CASE(TYPE)                            \
-  case Type::TYPE: {                                    \
-    return value_converter_->Extend(obj, value_length); \
+#define LIST_SLOW_CASE(TYPE_ID)                               \
+  case Type::TYPE_ID: {                                       \
+    return Extend(this->value_converter_.get(), value, size); \
   }

Review comment:
       Since these macros are only used in PyListConverter::AppendNdarray, please #define and #undef them inside that scope

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,296 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename BaseConverter, template <typename...> class ConverterTrait>
+static Result<std::shared_ptr<BaseConverter>> MakeConverter(
+    std::shared_ptr<DataType> type, typename BaseConverter::OptionsType options,
+    MemoryPool* pool);
+
+template <typename Input, typename Options>
+class Converter {
+ public:
+  using Self = Converter<Input, Options>;
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  Status Construct(std::shared_ptr<DataType> type, OptionsType options,
+                   MemoryPool* pool) {
+    type_ = std::move(type);
+    options_ = std::move(options);
+    return Init(pool);
+  }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>>& children() const { return children_; }
+
+  Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  virtual Status Init(MemoryPool* pool) { return Status::OK(); }
+
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+ protected:
+  Status Init(MemoryPool* pool) override {
+    this->builder_ = std::make_shared<BuilderType>(this->type_, pool);
+    this->primitive_type_ = checked_cast<const T*>(this->type_.get());
+    this->primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter, template <typename...> class ConverterTrait>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+  using ConverterType = typename ConverterTrait<T>::type;
+
+ protected:
+  Status Init(MemoryPool* pool) override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    ARROW_ASSIGN_OR_RAISE(value_converter_,
+                          (MakeConverter<BaseConverter, ConverterTrait>(
+                              list_type_->value_type(), this->options_, pool)));
+    this->builder_ =
+        std::make_shared<BuilderType>(pool, value_converter_->builder(), this->type_);
+    this->children_ = {value_converter_};
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter, template <typename...> class ConverterTrait>
+class StructConverter : public BaseConverter {
+ public:
+  using ConverterType = typename ConverterTrait<StructType>::type;
+
+ protected:
+  Status Init(MemoryPool* pool) override {
+    std::shared_ptr<BaseConverter> child_converter;
+    std::vector<std::shared_ptr<ArrayBuilder>> child_builders;
+
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    for (const auto& field : struct_type_->fields()) {
+      ARROW_ASSIGN_OR_RAISE(child_converter,
+                            (MakeConverter<BaseConverter, ConverterTrait>(
+                                field->type(), this->options_, pool)));
+      child_builders.push_back(child_converter->builder());
+      this->children_.push_back(std::move(child_converter));
+    }
+
+    this->builder_ =
+        std::make_shared<StructBuilder>(this->type_, pool, std::move(child_builders));
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+
+    return Status::OK();
+  }
+
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+ protected:
+  Status Init(MemoryPool* pool) override {
+    std::unique_ptr<ArrayBuilder> builder;
+    ARROW_RETURN_NOT_OK(MakeDictionaryBuilder(pool, this->type_, NULLPTR, &builder));
+    this->builder_ = std::move(builder);
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename BaseConverter, template <typename...> class ConverterTrait>
+struct MakeConverterImpl {
+  template <typename T, typename ConverterType = typename ConverterTrait<T>::type>
+  Status Visit(const T&) {

Review comment:
       MSVC is [failing to SFINAE](https://ci.appveyor.com/project/ApacheSoftwareFoundation/arrow/builds/35323354/job/0vyq23tsq8a7kwnt#L1015) for `T=DictionaryType`. We'll need to give it some help by changing the dictionary case a bit. Might work:
   
   ```c++
   template <>
   struct PyConverterTrait<DictionaryType> {
     template <typename T>
     using dictionary_type = PyDictionaryConverter<T>;
   };
   ```




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/array.pxi
##########
@@ -21,28 +21,28 @@ import warnings
 
 cdef _sequence_to_array(object sequence, object mask, object size,
                         DataType type, CMemoryPool* pool, c_bool from_pandas):
-    cdef int64_t c_size
-    cdef PyConversionOptions options
+    cdef:
+        int64_t c_size
+        PyConversionOptions options
+        shared_ptr[CChunkedArray] chunked
 
     if type is not None:
         options.type = type.sp_type
 
     if size is not None:
         options.size = size
 
-    options.pool = pool
     options.from_pandas = from_pandas
-    options.ignore_timezone = os.environ.get('PYARROW_IGNORE_TIMEZONE', False)

Review comment:
       Added a [unittest](https://github.com/apache/arrow/pull/8088/commits/29aa420f020233f29fd8ff225c58717e6c17b634#diff-7fddb4a3590947db6d84787eca4d80faR969-R998) for it, please check its validity.




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

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



[GitHub] [arrow] github-actions[bot] commented on pull request #8088: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API [WIP]

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


   <!--
     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.
   -->
   
   Thanks for opening a pull request!
   
   Could you open an issue for this pull request on JIRA?
   https://issues.apache.org/jira/browse/ARROW
   
   Then could you also rename pull request title in the following format?
   
       ARROW-${JIRA_ID}: [${COMPONENT}] ${SUMMARY}
   
   See also:
   
     * [Other pull requests](https://github.com/apache/arrow/pulls/)
     * [Contribution Guidelines - How to contribute patches](https://arrow.apache.org/docs/developers/contributing.html#how-to-contribute-patches)
   


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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/tests/test_convert_builtin.py
##########
@@ -1513,6 +1519,108 @@ def test_struct_from_tuples():
             pa.array([tup], type=ty)
 
 
+def test_struct_from_list_of_pairs():
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 6), ('b', 'bar'), ('c', False)],
+        None
+    ]
+    arr = pa.array(data, type=ty)
+    assert arr.to_pylist() == [
+        {'a': 5, 'b': 'foo', 'c': True},
+        {'a': 6, 'b': 'bar', 'c': False},
+        None
+    ]
+
+    # test with duplicated field names
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('a', pa.string()),
+        pa.field('b', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('a', 'foo'), ('b', True)],
+        [('a', 6), ('a', 'bar'), ('b', False)],
+    ]
+    arr = pa.array(data, type=ty)
+    with pytest.raises(KeyError):
+        # TODO(kszucs): ARROW-9997
+        arr.to_pylist()
+
+    # test with empty elements
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [],
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 2), ('b', 'baz')],
+        [('a', 1), ('b', 'bar'), ('c', False), ('d', 'julia')],

Review comment:
       It tolerates extra items (similarly when converting from dicts). I can restrict it if you think that would be more appropiate. We may want to alter this behavior with specific conversion flags.




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -329,985 +302,596 @@ struct ValueConverter<DurationType> {
         default:
           return Status::UnknownError("Invalid time unit");
       }
+    } else if (PyArray_CheckAnyScalarExact(obj)) {
+      // validate that the numpy scalar has np.datetime64 dtype
+      std::shared_ptr<DataType> numpy_type;
+      RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &numpy_type));
+      if (!numpy_type->Equals(*type)) {
+        return Status::NotImplemented("Expected np.timedelta64 but got: ",
+                                      numpy_type->ToString());
+      }
+      return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
     } else {
       RETURN_NOT_OK(internal::CIntFromPython(obj, &value));
     }
     return value;
   }
 
-  static inline Result<int64_t> FromNumpy(PyObject* obj, TimeUnit::type unit) {
-    // validate that the numpy scalar has np.timedelta64 dtype
-    std::shared_ptr<DataType> type;
-    RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &type));
-    if (type->id() != DurationType::type_id) {
-      // TODO(kszucs): the message should highlight the received numpy dtype
-      return Status::Invalid("Expected np.timedelta64 but got: ", type->ToString());
-    }
-    // validate that the time units are matching
-    if (unit != checked_cast<const DurationType&>(*type).unit()) {
-      return Status::NotImplemented(
-          "Cannot convert NumPy np.timedelta64 objects with differing unit");
-    }
-    // convert the numpy value
-    return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_any_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    return std::move(view);
-  }
-};
+  // The binary-like intermediate representation is PyBytesView because it keeps temporary
+  // python objects alive (non-contiguous memoryview) and stores whether the original
+  // object was unicode encoded or not, which is used for unicode -> bytes coersion if
+  // there is a non-unicode object observed.
 
-template <typename Type>
-struct ValueConverter<Type, enable_if_string_like<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    // strict conversion, force output to be unicode / utf8 and validate that
-    // any binary values are utf8
-    bool is_utf8 = false;
-    PyBytesView view;
-
-    RETURN_NOT_OK(view.FromString(obj, &is_utf8));
-    if (!is_utf8) {
-      return internal::InvalidValue(obj, "was not a utf8 string");
-    }
-    return std::move(view);
+  static Result<PyBytesView> Convert(const BaseBinaryType*, const O&, I obj) {
+    return PyBytesView::FromString(obj);
   }
 
-  static inline Result<PyBytesView> FromPython(PyObject* obj, bool* is_utf8) {
-    PyBytesView view;
-
-    // Non-strict conversion; keep track of whether values are unicode or bytes
-    if (PyUnicode_Check(obj)) {
-      *is_utf8 = true;
-      RETURN_NOT_OK(view.FromUnicode(obj));
+  static Result<PyBytesView> Convert(const FixedSizeBinaryType* type, const O&, I obj) {
+    ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj));
+    if (ARROW_PREDICT_TRUE(view.size == type->byte_width())) {
+      return std::move(view);
     } else {
-      // If not unicode or bytes, FromBinary will error
-      *is_utf8 = false;
-      RETURN_NOT_OK(view.FromBinary(obj));
-    }
-    return std::move(view);
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_fixed_size_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj, int32_t byte_width) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    if (ARROW_PREDICT_FALSE(view.size != byte_width)) {
       std::stringstream ss;
-      ss << "expected to be length " << byte_width << " was " << view.size;
+      ss << "expected to be length " << type->byte_width() << " was " << view.size;
       return internal::InvalidValue(obj, ss.str());
-    } else {
-      return std::move(view);
     }
   }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter base and CRTP "middle" subclasses
-
-class SeqConverter;
-
-// Forward-declare converter factory
-Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
-                    bool strict_conversions, bool ignore_timezone,
-                    std::unique_ptr<SeqConverter>* out);
-
-// Marshal Python sequence (list, tuple, etc.) to Arrow array
-class SeqConverter {
- public:
-  virtual ~SeqConverter() = default;
-
-  // Initialize the sequence converter with an ArrayBuilder created
-  // externally. The reason for this interface is that we have
-  // arrow::MakeBuilder which also creates child builders for nested types, so
-  // we have to pass in the child builders to child SeqConverter in the case of
-  // converting Python objects to Arrow nested types
-  virtual Status Init(ArrayBuilder* builder) = 0;
 
-  // Append a single null value to the builder
-  virtual Status AppendNull() = 0;
-
-  // Append a valid value
-  virtual Status AppendValue(PyObject* seq) = 0;
-
-  // Append a single python object handling Null values
-  virtual Status Append(PyObject* seq) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status Extend(PyObject* seq, int64_t size) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status ExtendMasked(PyObject* seq, PyObject* mask, int64_t size) = 0;
-
-  virtual Status Close() {
-    if (chunks_.size() == 0 || builder_->length() > 0) {
-      std::shared_ptr<Array> last_chunk;
-      RETURN_NOT_OK(builder_->Finish(&last_chunk));
-      chunks_.emplace_back(std::move(last_chunk));
+  template <typename T>
+  static enable_if_string<T, Result<PyBytesView>> Convert(const T*, const O& options,
+                                                          I obj) {
+    if (options.strict) {
+      // Strict conversion, force output to be unicode / utf8 and validate that
+      // any binary values are utf8
+      ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj, true));
+      if (!view.is_utf8) {
+        return internal::InvalidValue(obj, "was not a utf8 string");
+      }
+      return std::move(view);
+    } else {
+      // Non-strict conversion; keep track of whether values are unicode or bytes
+      return PyBytesView::FromString(obj);
     }
-    return Status::OK();
   }
 
-  virtual Status GetResult(std::shared_ptr<ChunkedArray>* out) {
-    // Still some accumulated data in the builder. If there are no chunks, we
-    // always call Finish to deal with the edge case where a size-0 sequence
-    // was converted with a specific output type, like array([], type=t)
-    RETURN_NOT_OK(Close());
-    *out = std::make_shared<ChunkedArray>(this->chunks_, builder_->type());
-    return Status::OK();
+  static Result<bool> Convert(const DataType* type, const O&, I obj) {
+    return Status::NotImplemented("PyValue::Convert is not implemented for type ", type);
   }
-
-  ArrayBuilder* builder() const { return builder_; }
-
-  int num_chunks() const { return static_cast<int>(chunks_.size()); }
-
- protected:
-  ArrayBuilder* builder_;
-  bool unfinished_builder_;
-  std::vector<std::shared_ptr<Array>> chunks_;
 };
 
-template <typename Type, NullCoding null_coding = NullCoding::NONE_ONLY>
-class TypedConverter : public SeqConverter {
- public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  Status Init(ArrayBuilder* builder) override {
-    builder_ = builder;
-    DCHECK_NE(builder_, nullptr);
-    typed_builder_ = checked_cast<BuilderType*>(builder);
-    return Status::OK();
-  }
-
-  // Append a missing item (default implementation)
-  Status AppendNull() override { return this->typed_builder_->AppendNull(); }
-
-  // Append null if the obj is None or pandas null otherwise the valid value
-  Status Append(PyObject* obj) override {
-    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
-  }
-
-  Status Extend(PyObject* obj, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequence(
-        obj, [this](PyObject* item, bool* /* unused */) { return this->Append(item); });
-  }
+template <typename T>
+Status Extend(T* converter, PyObject* values, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequence(values, [converter](PyObject* item, bool* /* unused */) {
+    return converter->Append(item);
+  });
+}
 
-  Status ExtendMasked(PyObject* obj, PyObject* mask, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequenceMasked(
-        obj, mask, [this](PyObject* item, bool is_masked, bool* /* unused */) {
-          if (is_masked) {
-            return this->AppendNull();
-          } else {
-            // This will also apply the null-checking convention in the event
-            // that the value is not masked
-            return this->Append(item);  // perhaps use AppendValue instead?
-          }
-        });
-  }
+// Convert and append a sequence of values masked with a numpy array
+template <typename T>
+Status ExtendMasked(T* converter, PyObject* values, PyObject* mask, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequenceMasked(
+      values, mask, [converter](PyObject* item, bool is_masked, bool* /* unused */) {
+        if (is_masked) {
+          return converter->AppendNull();
+        } else {
+          // This will also apply the null-checking convention in the event
+          // that the value is not masked
+          return converter->Append(item);  // perhaps use AppendValue instead?
+        }
+      });
+}
 
- protected:
-  BuilderType* typed_builder_;
-};
+// The base Converter class is a mixin with predefined behavior and constructors.
+class PyConverter : public Converter<PyObject*, PyConversionOptions, PyConverter> {};

Review comment:
       Updated.

##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -329,985 +302,596 @@ struct ValueConverter<DurationType> {
         default:
           return Status::UnknownError("Invalid time unit");
       }
+    } else if (PyArray_CheckAnyScalarExact(obj)) {
+      // validate that the numpy scalar has np.datetime64 dtype
+      std::shared_ptr<DataType> numpy_type;
+      RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &numpy_type));
+      if (!numpy_type->Equals(*type)) {
+        return Status::NotImplemented("Expected np.timedelta64 but got: ",
+                                      numpy_type->ToString());
+      }
+      return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
     } else {
       RETURN_NOT_OK(internal::CIntFromPython(obj, &value));
     }
     return value;
   }
 
-  static inline Result<int64_t> FromNumpy(PyObject* obj, TimeUnit::type unit) {
-    // validate that the numpy scalar has np.timedelta64 dtype
-    std::shared_ptr<DataType> type;
-    RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &type));
-    if (type->id() != DurationType::type_id) {
-      // TODO(kszucs): the message should highlight the received numpy dtype
-      return Status::Invalid("Expected np.timedelta64 but got: ", type->ToString());
-    }
-    // validate that the time units are matching
-    if (unit != checked_cast<const DurationType&>(*type).unit()) {
-      return Status::NotImplemented(
-          "Cannot convert NumPy np.timedelta64 objects with differing unit");
-    }
-    // convert the numpy value
-    return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_any_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    return std::move(view);
-  }
-};
+  // The binary-like intermediate representation is PyBytesView because it keeps temporary
+  // python objects alive (non-contiguous memoryview) and stores whether the original
+  // object was unicode encoded or not, which is used for unicode -> bytes coersion if
+  // there is a non-unicode object observed.
 
-template <typename Type>
-struct ValueConverter<Type, enable_if_string_like<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    // strict conversion, force output to be unicode / utf8 and validate that
-    // any binary values are utf8
-    bool is_utf8 = false;
-    PyBytesView view;
-
-    RETURN_NOT_OK(view.FromString(obj, &is_utf8));
-    if (!is_utf8) {
-      return internal::InvalidValue(obj, "was not a utf8 string");
-    }
-    return std::move(view);
+  static Result<PyBytesView> Convert(const BaseBinaryType*, const O&, I obj) {
+    return PyBytesView::FromString(obj);
   }
 
-  static inline Result<PyBytesView> FromPython(PyObject* obj, bool* is_utf8) {
-    PyBytesView view;
-
-    // Non-strict conversion; keep track of whether values are unicode or bytes
-    if (PyUnicode_Check(obj)) {
-      *is_utf8 = true;
-      RETURN_NOT_OK(view.FromUnicode(obj));
+  static Result<PyBytesView> Convert(const FixedSizeBinaryType* type, const O&, I obj) {
+    ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj));
+    if (ARROW_PREDICT_TRUE(view.size == type->byte_width())) {
+      return std::move(view);
     } else {
-      // If not unicode or bytes, FromBinary will error
-      *is_utf8 = false;
-      RETURN_NOT_OK(view.FromBinary(obj));
-    }
-    return std::move(view);
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_fixed_size_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj, int32_t byte_width) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    if (ARROW_PREDICT_FALSE(view.size != byte_width)) {
       std::stringstream ss;
-      ss << "expected to be length " << byte_width << " was " << view.size;
+      ss << "expected to be length " << type->byte_width() << " was " << view.size;
       return internal::InvalidValue(obj, ss.str());
-    } else {
-      return std::move(view);
     }
   }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter base and CRTP "middle" subclasses
-
-class SeqConverter;
-
-// Forward-declare converter factory
-Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
-                    bool strict_conversions, bool ignore_timezone,
-                    std::unique_ptr<SeqConverter>* out);
-
-// Marshal Python sequence (list, tuple, etc.) to Arrow array
-class SeqConverter {
- public:
-  virtual ~SeqConverter() = default;
-
-  // Initialize the sequence converter with an ArrayBuilder created
-  // externally. The reason for this interface is that we have
-  // arrow::MakeBuilder which also creates child builders for nested types, so
-  // we have to pass in the child builders to child SeqConverter in the case of
-  // converting Python objects to Arrow nested types
-  virtual Status Init(ArrayBuilder* builder) = 0;
 
-  // Append a single null value to the builder
-  virtual Status AppendNull() = 0;
-
-  // Append a valid value
-  virtual Status AppendValue(PyObject* seq) = 0;
-
-  // Append a single python object handling Null values
-  virtual Status Append(PyObject* seq) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status Extend(PyObject* seq, int64_t size) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status ExtendMasked(PyObject* seq, PyObject* mask, int64_t size) = 0;
-
-  virtual Status Close() {
-    if (chunks_.size() == 0 || builder_->length() > 0) {
-      std::shared_ptr<Array> last_chunk;
-      RETURN_NOT_OK(builder_->Finish(&last_chunk));
-      chunks_.emplace_back(std::move(last_chunk));
+  template <typename T>
+  static enable_if_string<T, Result<PyBytesView>> Convert(const T*, const O& options,
+                                                          I obj) {
+    if (options.strict) {
+      // Strict conversion, force output to be unicode / utf8 and validate that
+      // any binary values are utf8
+      ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj, true));
+      if (!view.is_utf8) {
+        return internal::InvalidValue(obj, "was not a utf8 string");
+      }
+      return std::move(view);
+    } else {
+      // Non-strict conversion; keep track of whether values are unicode or bytes
+      return PyBytesView::FromString(obj);
     }
-    return Status::OK();
   }
 
-  virtual Status GetResult(std::shared_ptr<ChunkedArray>* out) {
-    // Still some accumulated data in the builder. If there are no chunks, we
-    // always call Finish to deal with the edge case where a size-0 sequence
-    // was converted with a specific output type, like array([], type=t)
-    RETURN_NOT_OK(Close());
-    *out = std::make_shared<ChunkedArray>(this->chunks_, builder_->type());
-    return Status::OK();
+  static Result<bool> Convert(const DataType* type, const O&, I obj) {
+    return Status::NotImplemented("PyValue::Convert is not implemented for type ", type);
   }
-
-  ArrayBuilder* builder() const { return builder_; }
-
-  int num_chunks() const { return static_cast<int>(chunks_.size()); }
-
- protected:
-  ArrayBuilder* builder_;
-  bool unfinished_builder_;
-  std::vector<std::shared_ptr<Array>> chunks_;
 };
 
-template <typename Type, NullCoding null_coding = NullCoding::NONE_ONLY>
-class TypedConverter : public SeqConverter {
- public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  Status Init(ArrayBuilder* builder) override {
-    builder_ = builder;
-    DCHECK_NE(builder_, nullptr);
-    typed_builder_ = checked_cast<BuilderType*>(builder);
-    return Status::OK();
-  }
-
-  // Append a missing item (default implementation)
-  Status AppendNull() override { return this->typed_builder_->AppendNull(); }
-
-  // Append null if the obj is None or pandas null otherwise the valid value
-  Status Append(PyObject* obj) override {
-    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
-  }
-
-  Status Extend(PyObject* obj, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequence(
-        obj, [this](PyObject* item, bool* /* unused */) { return this->Append(item); });
-  }
+template <typename T>
+Status Extend(T* converter, PyObject* values, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequence(values, [converter](PyObject* item, bool* /* unused */) {
+    return converter->Append(item);
+  });
+}
 
-  Status ExtendMasked(PyObject* obj, PyObject* mask, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequenceMasked(
-        obj, mask, [this](PyObject* item, bool is_masked, bool* /* unused */) {
-          if (is_masked) {
-            return this->AppendNull();
-          } else {
-            // This will also apply the null-checking convention in the event
-            // that the value is not masked
-            return this->Append(item);  // perhaps use AppendValue instead?
-          }
-        });
-  }
+// Convert and append a sequence of values masked with a numpy array
+template <typename T>
+Status ExtendMasked(T* converter, PyObject* values, PyObject* mask, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequenceMasked(
+      values, mask, [converter](PyObject* item, bool is_masked, bool* /* unused */) {
+        if (is_masked) {
+          return converter->AppendNull();
+        } else {
+          // This will also apply the null-checking convention in the event
+          // that the value is not masked
+          return converter->Append(item);  // perhaps use AppendValue instead?
+        }
+      });
+}
 
- protected:
-  BuilderType* typed_builder_;
-};
+// The base Converter class is a mixin with predefined behavior and constructors.
+class PyConverter : public Converter<PyObject*, PyConversionOptions, PyConverter> {};
 
-// ----------------------------------------------------------------------
-// Sequence converter for null type
+template <typename T, typename Enable = void>
+class PyPrimitiveConverter;
 
-template <NullCoding null_coding>
-class NullConverter : public TypedConverter<NullType, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_null_type<T>::value || is_boolean_type<T>::value ||
+                   is_number_type<T>::value || is_decimal_type<T>::value ||
+                   is_date_type<T>::value || is_time_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  Status AppendValue(PyObject* obj) override {
-    return internal::InvalidValue(obj, "converting to null type");
-  }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter template for primitive (integer and floating point bool) types
-
-template <typename Type, NullCoding null_coding>
-class PrimitiveConverter : public TypedConverter<Type, null_coding> {
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value, ValueConverter<Type>::FromPython(obj));
-    return this->typed_builder_->Append(value);
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      return this->primitive_builder_->Append(converted);
+    }
   }
 };
 
-// ----------------------------------------------------------------------
-// Sequence converters for temporal types
-
-template <typename Type, NullCoding null_coding>
-class TimeConverter : public TypedConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_timestamp_type<T>::value || is_duration_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  explicit TimeConverter(TimeUnit::type unit, bool ignore_timezone)
-      : unit_(unit), ignore_timezone_(ignore_timezone) {}
-
-  // TODO(kszucs): support numpy values for date and time converters
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value,
-                          ValueConverter<Type>::FromPython(obj, unit_, ignore_timezone_));
-    return this->typed_builder_->Append(value);
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      // Numpy NaT sentinels can be checked after the conversion
+      if (PyArray_CheckAnyScalarExact(value) &&
+          PyValue::IsNaT(this->primitive_type_, converted)) {
+        return this->primitive_builder_->AppendNull();
+      } else {
+        return this->primitive_builder_->Append(converted);
+      }
+    }
   }
-
- protected:
-  TimeUnit::type unit_;
-  bool ignore_timezone_;
 };
 
-// TODO(kszucs): move it to the type_traits
 template <typename T>
-struct NumpyType {};
-
-template <>
-struct NumpyType<TimestampType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_DATETIME>::isnull(v);
-  }
-};
-
-template <>
-struct NumpyType<DurationType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_TIMEDELTA>::isnull(v);
-  }
-};
-
-template <typename Type, NullCoding null_coding>
-class TemporalConverter : public TimeConverter<Type, null_coding> {
+class PyPrimitiveConverter<T, enable_if_binary<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using TimeConverter<Type, null_coding>::TimeConverter;
-
-  Status AppendValue(PyObject* obj) override {
-    int64_t value;
-    if (PyArray_CheckAnyScalarExact(obj)) {
-      // convert np.datetime64 / np.timedelta64 depending on Type
-      ARROW_ASSIGN_OR_RAISE(value, ValueConverter<Type>::FromNumpy(obj, this->unit_));
-      if (NumpyType<Type>::isnull(value)) {
-        // checks numpy NaT sentinel after conversion
-        return this->typed_builder_->AppendNull();
-      }
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
     } else {
       ARROW_ASSIGN_OR_RAISE(
-          value,
-          ValueConverter<Type>::FromPython(
-              obj, this->unit_, TimeConverter<Type, null_coding>::ignore_timezone_));
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
     }
-    return this->typed_builder_->Append(value);
   }
 };
 
-// ----------------------------------------------------------------------
-// Sequence converters for Binary, FixedSizeBinary, String
-
-template <typename Type, NullCoding null_coding>
-class BinaryLikeConverter : public TypedConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_string_like<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  inline Status AutoChunk(Py_ssize_t size) {
-    // did we reach the builder size limit?
-    if (ARROW_PREDICT_FALSE(this->typed_builder_->value_data_length() + size >
-                            BuilderType::memory_limit())) {
-      // builder would be full, so need to add a new chunk
-      std::shared_ptr<Array> chunk;
-      RETURN_NOT_OK(this->typed_builder_->Finish(&chunk));
-      this->chunks_.emplace_back(std::move(chunk));
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      if (!view.is_utf8) {
+        // observed binary value
+        observed_binary_ = true;
+      }
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
     }
-    return Status::OK();
   }
 
-  Status AppendString(const PyBytesView& view) {
-    // check that the value fits in the datatype
-    if (view.size > BuilderType::memory_limit()) {
-      return Status::Invalid("string too large for datatype");
+  Result<std::shared_ptr<Array>> ToArray() override {
+    ARROW_ASSIGN_OR_RAISE(auto array, (PrimitiveConverter<T, PyConverter>::ToArray()));
+    if (observed_binary_) {
+      // if we saw any non-unicode, cast results to BinaryArray
+      auto binary_type = TypeTraits<typename T::PhysicalType>::type_singleton();
+      return array->View(binary_type);
+    } else {
+      return array;
     }
-    DCHECK_GE(view.size, 0);
-
-    // create a new chunk if the value would overflow the builder
-    RETURN_NOT_OK(AutoChunk(view.size));
-
-    // now we can safely append the value to the builder
-    RETURN_NOT_OK(
-        this->typed_builder_->Append(::arrow::util::string_view(view.bytes, view.size)));
-
-    return Status::OK();
   }
 
  protected:
-  // Create a single instance of PyBytesView here to prevent unnecessary object
-  // creation/destruction
-  PyBytesView string_view_;
+  bool observed_binary_ = false;
 };
 
-template <typename Type, NullCoding null_coding>
-class BinaryConverter : public BinaryLikeConverter<Type, null_coding> {
- public:
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto view, ValueConverter<Type>::FromPython(obj));
-    return this->AppendString(view);
-  }
-};
+template <typename U, typename Enable = void>
+class PyDictionaryConverter;
 
-template <NullCoding null_coding>
-class FixedSizeBinaryConverter
-    : public BinaryLikeConverter<FixedSizeBinaryType, null_coding> {
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_c_type<U>>
+    : public DictionaryConverter<U, PyConverter> {
  public:
-  explicit FixedSizeBinaryConverter(int32_t byte_width) : byte_width_(byte_width) {}
-
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(
-        this->string_view_,
-        ValueConverter<FixedSizeBinaryType>::FromPython(obj, byte_width_));
-    return this->AppendString(this->string_view_);
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(auto converted,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(converted);
+    }
   }
-
- protected:
-  int32_t byte_width_;
 };
 
-// For String/UTF8, if strict_conversions enabled, we reject any non-UTF8,
-// otherwise we allow but return results as BinaryArray
-template <typename Type, bool Strict, NullCoding null_coding>
-class StringConverter : public BinaryLikeConverter<Type, null_coding> {
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_string_view<U>>
+    : public DictionaryConverter<U, PyConverter> {
  public:
-  StringConverter() : binary_count_(0) {}
-
-  Status AppendValue(PyObject* obj) override {
-    if (Strict) {
-      // raise if the object is not unicode or not an utf-8 encoded bytes
-      ARROW_ASSIGN_OR_RAISE(this->string_view_, ValueConverter<Type>::FromPython(obj));
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
     } else {
-      // keep track of whether values are unicode or bytes; if any bytes are
-      // observe, the result will be bytes
-      bool is_utf8;
-      ARROW_ASSIGN_OR_RAISE(this->string_view_,
-                            ValueConverter<Type>::FromPython(obj, &is_utf8));
-      if (!is_utf8) {
-        ++binary_count_;
-      }
+      ARROW_ASSIGN_OR_RAISE(auto view,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(util::string_view(view.bytes, view.size));
     }
-    return this->AppendString(this->string_view_);
   }
-
-  Status GetResult(std::shared_ptr<ChunkedArray>* out) override {
-    RETURN_NOT_OK(SeqConverter::GetResult(out));
-
-    // If we saw any non-unicode, cast results to BinaryArray
-    if (binary_count_) {
-      // We should have bailed out earlier
-      DCHECK(!Strict);
-      auto binary_type = TypeTraits<typename Type::PhysicalType>::type_singleton();
-      return (*out)->View(binary_type).Value(out);
-    }
-    return Status::OK();
-  }
-
- protected:
-  int64_t binary_count_;
 };
 
-// ----------------------------------------------------------------------
-// Convert lists (NumPy arrays containing lists or ndarrays as values)
-
 // If the value type does not match the expected NumPy dtype, then fall through
 // to a slower PySequence-based path
-#define LIST_FAST_CASE(TYPE, NUMPY_TYPE, ArrowType)            \
-  case Type::TYPE: {                                           \
-    if (PyArray_DESCR(arr)->type_num != NUMPY_TYPE) {          \
-      return value_converter_->Extend(obj, value_length);      \
-    }                                                          \
-    return AppendNdarrayTypedItem<NUMPY_TYPE, ArrowType>(arr); \
+#define LIST_FAST_CASE(TYPE_ID, TYPE, NUMPY_TYPE)               \
+  case Type::TYPE_ID: {                                         \
+    if (PyArray_DESCR(ndarray)->type_num != NUMPY_TYPE) {       \
+      return Extend(this->value_converter_.get(), value, size); \
+    }                                                           \
+    return AppendNdarrayTyped<TYPE, NUMPY_TYPE>(ndarray);       \
   }
 
 // Use internal::VisitSequence, fast for NPY_OBJECT but slower otherwise
-#define LIST_SLOW_CASE(TYPE)                            \
-  case Type::TYPE: {                                    \
-    return value_converter_->Extend(obj, value_length); \
+#define LIST_SLOW_CASE(TYPE_ID)                               \
+  case Type::TYPE_ID: {                                       \
+    return Extend(this->value_converter_.get(), value, size); \
   }
 
-// Base class for ListConverter and FixedSizeListConverter (to have both work with CRTP)
-template <typename TypeClass, NullCoding null_coding>
-class BaseListConverter : public TypedConverter<TypeClass, null_coding> {
+template <typename T>
+class PyListConverter : public ListConverter<T, PyConverter> {
  public:
-  using BuilderType = typename TypeTraits<TypeClass>::BuilderType;
-
-  explicit BaseListConverter(bool from_pandas, bool strict_conversions,
-                             bool ignore_timezone)
-      : from_pandas_(from_pandas),
-        strict_conversions_(strict_conversions),
-        ignore_timezone_(ignore_timezone) {}
-
-  Status Init(ArrayBuilder* builder) override {
-    this->builder_ = builder;
-    this->typed_builder_ = checked_cast<BuilderType*>(builder);
-
-    this->value_type_ = checked_cast<const TypeClass&>(*builder->type()).value_type();
-    RETURN_NOT_OK(GetConverter(value_type_, from_pandas_, strict_conversions_,
-                               ignore_timezone_, &value_converter_));
-    return this->value_converter_->Init(this->typed_builder_->value_builder());
+  Status ValidateOverflow(const MapType*, int64_t size) { return Status::OK(); }
+
+  Status ValidateOverflow(const BaseListType*, int64_t size) {
+    return this->list_builder_->ValidateOverflow(size);
   }
 
-  template <int NUMPY_TYPE, typename Type>
-  Status AppendNdarrayTypedItem(PyArrayObject* arr) {
-    using traits = internal::npy_traits<NUMPY_TYPE>;
-    using T = typename traits::value_type;
-    using ValueBuilderType = typename TypeTraits<Type>::BuilderType;
+  Status ValidateBuilder(const MapType*) {
+    if (this->list_builder_->key_builder()->null_count() > 0) {
+      return Status::Invalid("Invalid Map: key field can not contain null values");
+    } else {
+      return Status::OK();
+    }
+  }
 
-    const bool null_sentinels_possible =
-        // Always treat Numpy's NaT as null
-        NUMPY_TYPE == NPY_DATETIME || NUMPY_TYPE == NPY_TIMEDELTA ||
-        // Observing pandas's null sentinels
-        (from_pandas_ && traits::supports_nulls);
+  Status ValidateBuilder(const BaseListType*) { return Status::OK(); }
 
-    auto child_builder = checked_cast<ValueBuilderType*>(value_converter_->builder());
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->list_builder_->AppendNull();
+    }
 
-    // TODO(wesm): Vector append when not strided
-    Ndarray1DIndexer<T> values(arr);
-    if (null_sentinels_possible) {
-      for (int64_t i = 0; i < values.size(); ++i) {
-        if (traits::isnull(values[i])) {
-          RETURN_NOT_OK(child_builder->AppendNull());
-        } else {
-          RETURN_NOT_OK(child_builder->Append(values[i]));
-        }
-      }
+    RETURN_NOT_OK(this->list_builder_->Append());
+    if (PyArray_Check(value)) {
+      RETURN_NOT_OK(AppendNdarray(value));
+    } else if (PySequence_Check(value)) {
+      RETURN_NOT_OK(AppendSequence(value));
     } else {
-      for (int64_t i = 0; i < values.size(); ++i) {
-        RETURN_NOT_OK(child_builder->Append(values[i]));
-      }
+      return internal::InvalidType(
+          value, "was not a sequence or recognized null for conversion to list type");
     }
-    return Status::OK();
+
+    return ValidateBuilder(this->list_type_);
   }
 
-  Status AppendNdarrayItem(PyObject* obj) {
-    PyArrayObject* arr = reinterpret_cast<PyArrayObject*>(obj);
+  Status AppendSequence(PyObject* value) {
+    int64_t size = static_cast<int64_t>(PySequence_Size(value));
+    RETURN_NOT_OK(ValidateOverflow(this->list_type_, size));
+    return Extend(this->value_converter_.get(), value, size);
+  }
 
-    if (PyArray_NDIM(arr) != 1) {
+  Status AppendNdarray(PyObject* value) {
+    PyArrayObject* ndarray = reinterpret_cast<PyArrayObject*>(value);
+    if (PyArray_NDIM(ndarray) != 1) {
       return Status::Invalid("Can only convert 1-dimensional array values");
     }
+    const int64_t size = PyArray_SIZE(ndarray);
+    RETURN_NOT_OK(ValidateOverflow(this->list_type_, size));
 
-    const int64_t value_length = PyArray_SIZE(arr);
-
-    switch (value_type_->id()) {
+    const auto value_type = this->value_converter_->builder()->type();
+    switch (value_type->id()) {
       LIST_SLOW_CASE(NA)
-      LIST_FAST_CASE(UINT8, NPY_UINT8, UInt8Type)
-      LIST_FAST_CASE(INT8, NPY_INT8, Int8Type)
-      LIST_FAST_CASE(UINT16, NPY_UINT16, UInt16Type)
-      LIST_FAST_CASE(INT16, NPY_INT16, Int16Type)
-      LIST_FAST_CASE(UINT32, NPY_UINT32, UInt32Type)
-      LIST_FAST_CASE(INT32, NPY_INT32, Int32Type)
-      LIST_FAST_CASE(UINT64, NPY_UINT64, UInt64Type)
-      LIST_FAST_CASE(INT64, NPY_INT64, Int64Type)
+      LIST_FAST_CASE(UINT8, UInt8Type, NPY_UINT8)
+      LIST_FAST_CASE(INT8, Int8Type, NPY_INT8)
+      LIST_FAST_CASE(UINT16, UInt16Type, NPY_UINT16)
+      LIST_FAST_CASE(INT16, Int16Type, NPY_INT16)
+      LIST_FAST_CASE(UINT32, UInt32Type, NPY_UINT32)
+      LIST_FAST_CASE(INT32, Int32Type, NPY_INT32)
+      LIST_FAST_CASE(UINT64, UInt64Type, NPY_UINT64)
+      LIST_FAST_CASE(INT64, Int64Type, NPY_INT64)
+      LIST_FAST_CASE(HALF_FLOAT, HalfFloatType, NPY_FLOAT16)
+      LIST_FAST_CASE(FLOAT, FloatType, NPY_FLOAT)
+      LIST_FAST_CASE(DOUBLE, DoubleType, NPY_DOUBLE)
+      LIST_FAST_CASE(TIMESTAMP, TimestampType, NPY_DATETIME)
+      LIST_FAST_CASE(DURATION, DurationType, NPY_TIMEDELTA)
       LIST_SLOW_CASE(DATE32)
       LIST_SLOW_CASE(DATE64)
       LIST_SLOW_CASE(TIME32)
       LIST_SLOW_CASE(TIME64)
-      LIST_FAST_CASE(TIMESTAMP, NPY_DATETIME, TimestampType)
-      LIST_FAST_CASE(DURATION, NPY_TIMEDELTA, DurationType)
-      LIST_FAST_CASE(HALF_FLOAT, NPY_FLOAT16, HalfFloatType)
-      LIST_FAST_CASE(FLOAT, NPY_FLOAT, FloatType)
-      LIST_FAST_CASE(DOUBLE, NPY_DOUBLE, DoubleType)
       LIST_SLOW_CASE(BINARY)
       LIST_SLOW_CASE(FIXED_SIZE_BINARY)
       LIST_SLOW_CASE(STRING)
       case Type::LIST: {
-        if (PyArray_DESCR(arr)->type_num != NPY_OBJECT) {
+        if (PyArray_DESCR(ndarray)->type_num != NPY_OBJECT) {
           return Status::Invalid(
-              "Can only convert list types from NumPy object "
-              "array input");
+              "Can only convert list types from NumPy object array input");
         }
-        return internal::VisitSequence(obj, [this](PyObject* item, bool*) {
-          return value_converter_->Append(item);
+        return internal::VisitSequence(value, [this](PyObject* item, bool*) {
+          return this->value_converter_->Append(item);
         });

Review comment:
       Updated.




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

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



[GitHub] [arrow] github-actions[bot] commented on pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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


   Revision: 79e8229d6828f7f63f80db75a4ef89af9deb03d3
   
   Submitted crossbow builds: [ursa-labs/crossbow @ actions-541](https://github.com/ursa-labs/crossbow/branches/all?query=actions-541)
   
   |Task|Status|
   |----|------|
   |test-conda-python-3.7-spark-branch-3.0|[![Github Actions](https://github.com/ursa-labs/crossbow/workflows/Crossbow/badge.svg?branch=actions-541-github-test-conda-python-3.7-spark-branch-3.0)](https://github.com/ursa-labs/crossbow/actions?query=branch:actions-541-github-test-conda-python-3.7-spark-branch-3.0)|
   |test-conda-python-3.8-spark-master|[![Github Actions](https://github.com/ursa-labs/crossbow/workflows/Crossbow/badge.svg?branch=actions-541-github-test-conda-python-3.8-spark-master)](https://github.com/ursa-labs/crossbow/actions?query=branch:actions-541-github-test-conda-python-3.8-spark-master)|


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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/array.pxi
##########
@@ -21,28 +21,28 @@ import warnings
 
 cdef _sequence_to_array(object sequence, object mask, object size,
                         DataType type, CMemoryPool* pool, c_bool from_pandas):
-    cdef int64_t c_size
-    cdef PyConversionOptions options
+    cdef:
+        int64_t c_size
+        PyConversionOptions options
+        shared_ptr[CChunkedArray] chunked
 
     if type is not None:
         options.type = type.sp_type
 
     if size is not None:
         options.size = size
 
-    options.pool = pool
     options.from_pandas = from_pandas
-    options.ignore_timezone = os.environ.get('PYARROW_IGNORE_TIMEZONE', False)

Review comment:
       It wasn't catched by any tests?

##########
File path: python/pyarrow/tests/test_convert_builtin.py
##########
@@ -1513,6 +1519,108 @@ def test_struct_from_tuples():
             pa.array([tup], type=ty)
 
 
+def test_struct_from_list_of_pairs():
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 6), ('b', 'bar'), ('c', False)],
+        None
+    ]
+    arr = pa.array(data, type=ty)
+    assert arr.to_pylist() == [
+        {'a': 5, 'b': 'foo', 'c': True},
+        {'a': 6, 'b': 'bar', 'c': False},
+        None
+    ]
+
+    # test with duplicated field names
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('a', pa.string()),
+        pa.field('b', pa.bool_())
+    ])
+    data = [
+        [('a', 5), ('a', 'foo'), ('b', True)],
+        [('a', 6), ('a', 'bar'), ('b', False)],
+    ]
+    arr = pa.array(data, type=ty)
+    with pytest.raises(KeyError):
+        # TODO(kszucs): ARROW-9997
+        arr.to_pylist()
+
+    # test with empty elements
+    ty = pa.struct([
+        pa.field('a', pa.int32()),
+        pa.field('b', pa.string()),
+        pa.field('c', pa.bool_())
+    ])
+    data = [
+        [],
+        [('a', 5), ('b', 'foo'), ('c', True)],
+        [('a', 2), ('b', 'baz')],
+        [('a', 1), ('b', 'bar'), ('c', False), ('d', 'julia')],

Review comment:
       Since the *order* matters when constructing from pairs (based on the test below), I would personally expect additional elements would also not be allowed (BTW, what actually happens if the additional element is not in the last position? Maybe add a test for that as well?). 
   While when constructing from dicts also the order does not matter I suppose? So this could be more tolerant
   
   




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

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



[GitHub] [arrow] bkietz commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/array/array_list_test.cc
##########
@@ -508,6 +534,8 @@ TYPED_TEST(TestListArray, ValidateOffsets) { this->TestValidateOffsets(); }
 
 TYPED_TEST(TestListArray, CornerCases) { this->TestCornerCases(); }
 
+TYPED_TEST(TestListArray, TestOverflowCheck) { this->TestOverflowCheck(); }

Review comment:
       ```suggestion
   TYPED_TEST(TestListArray, LARGE_MEMORY_TEST(TestOverflowCheck)) { this->TestOverflowCheck(); }
   ```




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/hashing.h
##########
@@ -851,6 +851,11 @@ struct HashTraits<T, enable_if_t<has_string_view<T>::value &&
   using MemoTableType = BinaryMemoTable<BinaryBuilder>;
 };
 
+template <>
+struct HashTraits<Decimal128Type> {
+  using MemoTableType = BinaryMemoTable<BinaryBuilder>;

Review comment:
       Sadly this is causing multiple compile errors due to missing `GetOrInsert` and `CopyFixedWidthValues` methods.




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/array.pxi
##########
@@ -21,28 +21,28 @@ import warnings
 
 cdef _sequence_to_array(object sequence, object mask, object size,
                         DataType type, CMemoryPool* pool, c_bool from_pandas):
-    cdef int64_t c_size
-    cdef PyConversionOptions options
+    cdef:
+        int64_t c_size
+        PyConversionOptions options
+        shared_ptr[CChunkedArray] chunked
 
     if type is not None:
         options.type = type.sp_type
 
     if size is not None:
         options.size = size
 
-    options.pool = pool
     options.from_pandas = from_pandas
-    options.ignore_timezone = os.environ.get('PYARROW_IGNORE_TIMEZONE', False)

Review comment:
       AFAIK It is only tested from the spark integration tests. I can add a unittest for 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.

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,348 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  virtual Status Initialize(std::shared_ptr<DataType> type,

Review comment:
       Updating.




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

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



[GitHub] [arrow] bkietz commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/python/python_to_arrow.h
##########
@@ -39,27 +39,26 @@ class Status;
 namespace py {
 
 struct PyConversionOptions {
-  PyConversionOptions() : type(NULLPTR), size(-1), pool(NULLPTR), from_pandas(false) {}
+  PyConversionOptions() : type(NULLPTR), size(-1), from_pandas(false) {}
 
   PyConversionOptions(const std::shared_ptr<DataType>& type, int64_t size,
                       MemoryPool* pool, bool from_pandas)
-      : type(type), size(size), pool(default_memory_pool()), from_pandas(from_pandas) {}
+      : type(type), size(size), from_pandas(from_pandas) {}
 
   // Set to null if to be inferred
   std::shared_ptr<DataType> type;
 
   // Default is -1: infer from data
   int64_t size;

Review comment:
       ```suggestion
     // Default is -1, which indicates the size should the same as the input sequence
     int64_t size = -1;
   ```

##########
File path: cpp/src/arrow/array/builder_binary.h
##########
@@ -317,16 +327,9 @@ class BaseBinaryBuilder : public ArrayBuilder {
   TypedBufferBuilder<offset_type> offsets_builder_;
   TypedBufferBuilder<uint8_t> value_data_builder_;
 
-  Status AppendOverflow(int64_t num_bytes) {
-    return Status::CapacityError("array cannot contain more than ", memory_limit(),
-                                 " bytes, have ", num_bytes);
-  }
-
   Status AppendNextOffset() {
     const int64_t num_bytes = value_data_builder_.length();
-    if (ARROW_PREDICT_FALSE(num_bytes > memory_limit())) {
-      return AppendOverflow(num_bytes);
-    }
+    ARROW_RETURN_NOT_OK(ValidateOverflow());

Review comment:
       ```suggestion
       ARROW_RETURN_NOT_OK(ValidateOverflow(num_bytes));
   ```
   ?

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter>
+struct MakeConverterImpl;
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  template <typename T>
+  using Primitive = PrimitiveConverter<T, Self>;
+  template <typename T>
+  using List = ListConverter<T, Self>;
+  template <typename T>
+  using Dictionary = DictionaryConverter<T, Self>;
+  using Struct = StructConverter<Self>;
+
+  static Result<std::shared_ptr<Self>> Make(std::shared_ptr<DataType> type,
+                                            MemoryPool* pool, OptionsType options) {
+    std::shared_ptr<Self> out;
+    MakeConverterImpl<Self> visitor = {type, pool, options, &out};
+    ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+    ARROW_RETURN_NOT_OK(out->Init());
+    return out;
+  }
+
+  virtual ~Converter() = default;
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>> children() const { return children_; }

Review comment:
       ```suggestion
     const std::vector<std::shared_ptr<Self>>& children() const { return children_; }
   ```

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter>
+struct MakeConverterImpl;
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  template <typename T>
+  using Primitive = PrimitiveConverter<T, Self>;
+  template <typename T>
+  using List = ListConverter<T, Self>;
+  template <typename T>
+  using Dictionary = DictionaryConverter<T, Self>;
+  using Struct = StructConverter<Self>;
+
+  static Result<std::shared_ptr<Self>> Make(std::shared_ptr<DataType> type,
+                                            MemoryPool* pool, OptionsType options) {
+    std::shared_ptr<Self> out;
+    MakeConverterImpl<Self> visitor = {type, pool, options, &out};
+    ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+    ARROW_RETURN_NOT_OK(out->Init());
+    return out;
+  }
+
+  virtual ~Converter() = default;
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>> children() const { return children_; }
+
+  virtual Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  virtual Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  friend struct MakeConverterImpl<Self>;
+
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+#define DICTIONARY_CASE(TYPE_ENUM, TYPE_CLASS)                          \
+  case Type::TYPE_ENUM:                                                 \
+    return Finish<typename Converter::template Dictionary<TYPE_CLASS>>( \
+        std::move(builder), {});                                        \
+    break;
+
+template <typename Converter>
+struct MakeConverterImpl {
+  Status Visit(const NullType& t) {
+    using BuilderType = typename TypeTraits<NullType>::BuilderType;
+    using ConverterType = typename Converter::template Primitive<NullType>;
+
+    auto builder = std::make_shared<BuilderType>(pool);
+    return Finish<ConverterType>(std::move(builder), {});
+  }
+
+  template <typename T>
+  enable_if_t<!is_nested_type<T>::value && !is_interval_type<T>::value &&
+                  !is_dictionary_type<T>::value && !is_extension_type<T>::value,
+              Status>
+  Visit(const T& t) {
+    using BuilderType = typename TypeTraits<T>::BuilderType;
+    using ConverterType = typename Converter::template Primitive<T>;
+
+    auto builder = std::make_shared<BuilderType>(type, pool);
+    return Finish<ConverterType>(std::move(builder), {});
+  }
+
+  template <typename T>
+  enable_if_t<is_list_like_type<T>::value && !std::is_same<T, MapType>::value, Status>
+  Visit(const T& t) {
+    using BuilderType = typename TypeTraits<T>::BuilderType;
+    using ConverterType = typename Converter::template List<T>;
+
+    ARROW_ASSIGN_OR_RAISE(auto child_converter,
+                          Converter::Make(t.value_type(), pool, options));
+    auto builder = std::make_shared<BuilderType>(pool, child_converter->builder(), type);
+    return Finish<ConverterType>(std::move(builder), {std::move(child_converter)});
+  }
+
+  Status Visit(const MapType& t) {
+    using ConverterType = typename Converter::template List<MapType>;
+
+    // TODO(kszucs): seems like builders not respect field nullability
+    std::vector<std::shared_ptr<Field>> struct_fields{t.key_field(), t.item_field()};
+    auto struct_type = std::make_shared<StructType>(struct_fields);
+    ARROW_ASSIGN_OR_RAISE(auto struct_converter,
+                          Converter::Make(struct_type, pool, options));
+
+    auto struct_builder = struct_converter->builder();
+    auto key_builder = struct_builder->child_builder(0);
+    auto item_builder = struct_builder->child_builder(1);
+    auto builder = std::make_shared<MapBuilder>(pool, key_builder, item_builder, type);
+
+    return Finish<ConverterType>(std::move(builder), {std::move(struct_converter)});
+  }
+
+  Status Visit(const DictionaryType& t) {
+    std::unique_ptr<ArrayBuilder> builder;
+    ARROW_RETURN_NOT_OK(MakeDictionaryBuilder(pool, type, NULLPTR, &builder));
+
+    switch (t.value_type()->id()) {
+      DICTIONARY_CASE(BOOL, BooleanType);
+      DICTIONARY_CASE(INT8, Int8Type);
+      DICTIONARY_CASE(INT16, Int16Type);
+      DICTIONARY_CASE(INT32, Int32Type);
+      DICTIONARY_CASE(INT64, Int64Type);
+      DICTIONARY_CASE(UINT8, UInt8Type);
+      DICTIONARY_CASE(UINT16, UInt16Type);
+      DICTIONARY_CASE(UINT32, UInt32Type);
+      DICTIONARY_CASE(UINT64, UInt64Type);
+      DICTIONARY_CASE(HALF_FLOAT, HalfFloatType);
+      DICTIONARY_CASE(FLOAT, FloatType);
+      DICTIONARY_CASE(DOUBLE, DoubleType);
+      DICTIONARY_CASE(DATE32, Date32Type);
+      DICTIONARY_CASE(DATE64, Date64Type);
+      DICTIONARY_CASE(BINARY, BinaryType);
+      DICTIONARY_CASE(STRING, StringType);
+      DICTIONARY_CASE(FIXED_SIZE_BINARY, FixedSizeBinaryType);
+      default:
+        return Status::NotImplemented("DictionaryArray converter for type ", t.ToString(),
+                                      " not implemented");
+    }
+  }
+
+  Status Visit(const StructType& t) {
+    using ConverterType = typename Converter::Struct;
+
+    std::shared_ptr<Converter> child_converter;
+    std::vector<std::shared_ptr<Converter>> child_converters;
+    std::vector<std::shared_ptr<ArrayBuilder>> child_builders;
+
+    for (const auto& field : t.fields()) {
+      ARROW_ASSIGN_OR_RAISE(child_converter,
+                            Converter::Make(field->type(), pool, options));
+
+      // TODO: use move
+      child_converters.push_back(child_converter);
+      child_builders.push_back(child_converter->builder());
+    }
+
+    auto builder = std::make_shared<StructBuilder>(type, pool, child_builders);
+    return Finish<ConverterType>(std::move(builder), std::move(child_converters));
+  }
+
+  Status Visit(const DataType& t) { return Status::NotImplemented(t.name()); }
+
+  template <typename ConverterType>
+  Status Finish(std::shared_ptr<ArrayBuilder> builder,
+                std::vector<std::shared_ptr<Converter>> children) {
+    auto converter = new ConverterType();
+    converter->type_ = std::move(type);
+    converter->builder_ = std::move(builder);
+    converter->options_ = options;
+    converter->children_ = std::move(children);
+    out->reset(converter);
+    return Status::OK();
+  }
+
+  const std::shared_ptr<DataType> type;

Review comment:
       ```suggestion
     std::shared_ptr<DataType> type;
   ```

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter>
+struct MakeConverterImpl;
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  template <typename T>
+  using Primitive = PrimitiveConverter<T, Self>;
+  template <typename T>
+  using List = ListConverter<T, Self>;
+  template <typename T>
+  using Dictionary = DictionaryConverter<T, Self>;
+  using Struct = StructConverter<Self>;
+
+  static Result<std::shared_ptr<Self>> Make(std::shared_ptr<DataType> type,
+                                            MemoryPool* pool, OptionsType options) {
+    std::shared_ptr<Self> out;
+    MakeConverterImpl<Self> visitor = {type, pool, options, &out};
+    ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+    ARROW_RETURN_NOT_OK(out->Init());
+    return out;
+  }
+
+  virtual ~Converter() = default;
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }

Review comment:
       Why do these have default implementations instead of being pure virtual?

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter>
+struct MakeConverterImpl;
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  template <typename T>
+  using Primitive = PrimitiveConverter<T, Self>;
+  template <typename T>
+  using List = ListConverter<T, Self>;
+  template <typename T>
+  using Dictionary = DictionaryConverter<T, Self>;
+  using Struct = StructConverter<Self>;
+
+  static Result<std::shared_ptr<Self>> Make(std::shared_ptr<DataType> type,
+                                            MemoryPool* pool, OptionsType options) {
+    std::shared_ptr<Self> out;
+    MakeConverterImpl<Self> visitor = {type, pool, options, &out};
+    ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+    ARROW_RETURN_NOT_OK(out->Init());
+    return out;
+  }
+
+  virtual ~Converter() = default;
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>> children() const { return children_; }
+
+  virtual Status Reserve(int64_t additional_capacity) {

Review comment:
       Maybe this can also be non virtual?

##########
File path: cpp/src/arrow/python/python_to_arrow.h
##########
@@ -39,27 +39,26 @@ class Status;
 namespace py {
 
 struct PyConversionOptions {
-  PyConversionOptions() : type(NULLPTR), size(-1), pool(NULLPTR), from_pandas(false) {}
+  PyConversionOptions() : type(NULLPTR), size(-1), from_pandas(false) {}

Review comment:
       ```suggestion
     PyConversionOptions() = default;
   ```

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>

Review comment:
       ```suggestion
   template <typename ValueType, typename BaseConverter>
   ```
   (since by the pattern established above one might expect `U` to be DictionaryType)

##########
File path: cpp/src/arrow/scalar_test.cc
##########
@@ -627,6 +627,9 @@ TEST(TestDictionaryScalar, Basics) {
     gamma.dictionary = dict;
 
     auto scalar_null = MakeNullScalar(ty);
+    auto& dict_scalar_null = checked_cast<DictionaryScalar&>(*scalar_null);
+    dict_scalar_null.value.dictionary = dict;

Review comment:
       nit: mutable references should be avoided where possible
   ```suggestion
       checked_cast<DictionaryScalar&>(*scalar_null).value.dictionary = dict;
   ```

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter>
+struct MakeConverterImpl;
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  template <typename T>
+  using Primitive = PrimitiveConverter<T, Self>;
+  template <typename T>
+  using List = ListConverter<T, Self>;
+  template <typename T>
+  using Dictionary = DictionaryConverter<T, Self>;
+  using Struct = StructConverter<Self>;
+
+  static Result<std::shared_ptr<Self>> Make(std::shared_ptr<DataType> type,
+                                            MemoryPool* pool, OptionsType options) {
+    std::shared_ptr<Self> out;
+    MakeConverterImpl<Self> visitor = {type, pool, options, &out};
+    ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+    ARROW_RETURN_NOT_OK(out->Init());
+    return out;
+  }
+
+  virtual ~Converter() = default;
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>> children() const { return children_; }
+
+  virtual Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  virtual Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  friend struct MakeConverterImpl<Self>;
+
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+#define DICTIONARY_CASE(TYPE_ENUM, TYPE_CLASS)                          \
+  case Type::TYPE_ENUM:                                                 \
+    return Finish<typename Converter::template Dictionary<TYPE_CLASS>>( \
+        std::move(builder), {});                                        \
+    break;
+
+template <typename Converter>
+struct MakeConverterImpl {
+  Status Visit(const NullType& t) {
+    using BuilderType = typename TypeTraits<NullType>::BuilderType;
+    using ConverterType = typename Converter::template Primitive<NullType>;
+
+    auto builder = std::make_shared<BuilderType>(pool);
+    return Finish<ConverterType>(std::move(builder), {});
+  }
+
+  template <typename T>
+  enable_if_t<!is_nested_type<T>::value && !is_interval_type<T>::value &&
+                  !is_dictionary_type<T>::value && !is_extension_type<T>::value,
+              Status>
+  Visit(const T& t) {
+    using BuilderType = typename TypeTraits<T>::BuilderType;
+    using ConverterType = typename Converter::template Primitive<T>;
+
+    auto builder = std::make_shared<BuilderType>(type, pool);
+    return Finish<ConverterType>(std::move(builder), {});
+  }
+
+  template <typename T>
+  enable_if_t<is_list_like_type<T>::value && !std::is_same<T, MapType>::value, Status>
+  Visit(const T& t) {
+    using BuilderType = typename TypeTraits<T>::BuilderType;
+    using ConverterType = typename Converter::template List<T>;
+
+    ARROW_ASSIGN_OR_RAISE(auto child_converter,
+                          Converter::Make(t.value_type(), pool, options));
+    auto builder = std::make_shared<BuilderType>(pool, child_converter->builder(), type);
+    return Finish<ConverterType>(std::move(builder), {std::move(child_converter)});
+  }
+
+  Status Visit(const MapType& t) {
+    using ConverterType = typename Converter::template List<MapType>;
+
+    // TODO(kszucs): seems like builders not respect field nullability
+    std::vector<std::shared_ptr<Field>> struct_fields{t.key_field(), t.item_field()};
+    auto struct_type = std::make_shared<StructType>(struct_fields);
+    ARROW_ASSIGN_OR_RAISE(auto struct_converter,
+                          Converter::Make(struct_type, pool, options));
+
+    auto struct_builder = struct_converter->builder();
+    auto key_builder = struct_builder->child_builder(0);
+    auto item_builder = struct_builder->child_builder(1);
+    auto builder = std::make_shared<MapBuilder>(pool, key_builder, item_builder, type);
+
+    return Finish<ConverterType>(std::move(builder), {std::move(struct_converter)});
+  }
+
+  Status Visit(const DictionaryType& t) {
+    std::unique_ptr<ArrayBuilder> builder;
+    ARROW_RETURN_NOT_OK(MakeDictionaryBuilder(pool, type, NULLPTR, &builder));
+
+    switch (t.value_type()->id()) {
+      DICTIONARY_CASE(BOOL, BooleanType);
+      DICTIONARY_CASE(INT8, Int8Type);
+      DICTIONARY_CASE(INT16, Int16Type);
+      DICTIONARY_CASE(INT32, Int32Type);
+      DICTIONARY_CASE(INT64, Int64Type);
+      DICTIONARY_CASE(UINT8, UInt8Type);
+      DICTIONARY_CASE(UINT16, UInt16Type);
+      DICTIONARY_CASE(UINT32, UInt32Type);
+      DICTIONARY_CASE(UINT64, UInt64Type);
+      DICTIONARY_CASE(HALF_FLOAT, HalfFloatType);
+      DICTIONARY_CASE(FLOAT, FloatType);
+      DICTIONARY_CASE(DOUBLE, DoubleType);
+      DICTIONARY_CASE(DATE32, Date32Type);
+      DICTIONARY_CASE(DATE64, Date64Type);
+      DICTIONARY_CASE(BINARY, BinaryType);
+      DICTIONARY_CASE(STRING, StringType);
+      DICTIONARY_CASE(FIXED_SIZE_BINARY, FixedSizeBinaryType);
+      default:
+        return Status::NotImplemented("DictionaryArray converter for type ", t.ToString(),
+                                      " not implemented");
+    }
+  }
+
+  Status Visit(const StructType& t) {
+    using ConverterType = typename Converter::Struct;
+
+    std::shared_ptr<Converter> child_converter;
+    std::vector<std::shared_ptr<Converter>> child_converters;
+    std::vector<std::shared_ptr<ArrayBuilder>> child_builders;
+
+    for (const auto& field : t.fields()) {
+      ARROW_ASSIGN_OR_RAISE(child_converter,
+                            Converter::Make(field->type(), pool, options));
+
+      // TODO: use move
+      child_converters.push_back(child_converter);
+      child_builders.push_back(child_converter->builder());
+    }
+
+    auto builder = std::make_shared<StructBuilder>(type, pool, child_builders);
+    return Finish<ConverterType>(std::move(builder), std::move(child_converters));
+  }
+
+  Status Visit(const DataType& t) { return Status::NotImplemented(t.name()); }
+
+  template <typename ConverterType>
+  Status Finish(std::shared_ptr<ArrayBuilder> builder,
+                std::vector<std::shared_ptr<Converter>> children) {
+    auto converter = new ConverterType();
+    converter->type_ = std::move(type);
+    converter->builder_ = std::move(builder);
+    converter->options_ = options;

Review comment:
       ```suggestion
       converter->options_ = std::move(options);
   ```

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter>
+struct MakeConverterImpl;
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  template <typename T>
+  using Primitive = PrimitiveConverter<T, Self>;
+  template <typename T>
+  using List = ListConverter<T, Self>;
+  template <typename T>
+  using Dictionary = DictionaryConverter<T, Self>;
+  using Struct = StructConverter<Self>;
+
+  static Result<std::shared_ptr<Self>> Make(std::shared_ptr<DataType> type,
+                                            MemoryPool* pool, OptionsType options) {
+    std::shared_ptr<Self> out;
+    MakeConverterImpl<Self> visitor = {type, pool, options, &out};
+    ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+    ARROW_RETURN_NOT_OK(out->Init());
+    return out;
+  }
+
+  virtual ~Converter() = default;
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>> children() const { return children_; }
+
+  virtual Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  virtual Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  friend struct MakeConverterImpl<Self>;
+
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+#define DICTIONARY_CASE(TYPE_ENUM, TYPE_CLASS)                          \
+  case Type::TYPE_ENUM:                                                 \
+    return Finish<typename Converter::template Dictionary<TYPE_CLASS>>( \
+        std::move(builder), {});                                        \
+    break;
+
+template <typename Converter>
+struct MakeConverterImpl {
+  Status Visit(const NullType& t) {
+    using BuilderType = typename TypeTraits<NullType>::BuilderType;
+    using ConverterType = typename Converter::template Primitive<NullType>;
+
+    auto builder = std::make_shared<BuilderType>(pool);
+    return Finish<ConverterType>(std::move(builder), {});
+  }
+
+  template <typename T>
+  enable_if_t<!is_nested_type<T>::value && !is_interval_type<T>::value &&
+                  !is_dictionary_type<T>::value && !is_extension_type<T>::value,
+              Status>
+  Visit(const T& t) {
+    using BuilderType = typename TypeTraits<T>::BuilderType;
+    using ConverterType = typename Converter::template Primitive<T>;
+
+    auto builder = std::make_shared<BuilderType>(type, pool);
+    return Finish<ConverterType>(std::move(builder), {});
+  }
+
+  template <typename T>
+  enable_if_t<is_list_like_type<T>::value && !std::is_same<T, MapType>::value, Status>
+  Visit(const T& t) {
+    using BuilderType = typename TypeTraits<T>::BuilderType;
+    using ConverterType = typename Converter::template List<T>;
+
+    ARROW_ASSIGN_OR_RAISE(auto child_converter,
+                          Converter::Make(t.value_type(), pool, options));
+    auto builder = std::make_shared<BuilderType>(pool, child_converter->builder(), type);
+    return Finish<ConverterType>(std::move(builder), {std::move(child_converter)});
+  }
+
+  Status Visit(const MapType& t) {
+    using ConverterType = typename Converter::template List<MapType>;
+
+    // TODO(kszucs): seems like builders not respect field nullability
+    std::vector<std::shared_ptr<Field>> struct_fields{t.key_field(), t.item_field()};
+    auto struct_type = std::make_shared<StructType>(struct_fields);
+    ARROW_ASSIGN_OR_RAISE(auto struct_converter,
+                          Converter::Make(struct_type, pool, options));
+
+    auto struct_builder = struct_converter->builder();
+    auto key_builder = struct_builder->child_builder(0);
+    auto item_builder = struct_builder->child_builder(1);
+    auto builder = std::make_shared<MapBuilder>(pool, key_builder, item_builder, type);
+
+    return Finish<ConverterType>(std::move(builder), {std::move(struct_converter)});
+  }
+
+  Status Visit(const DictionaryType& t) {
+    std::unique_ptr<ArrayBuilder> builder;
+    ARROW_RETURN_NOT_OK(MakeDictionaryBuilder(pool, type, NULLPTR, &builder));
+
+    switch (t.value_type()->id()) {
+      DICTIONARY_CASE(BOOL, BooleanType);
+      DICTIONARY_CASE(INT8, Int8Type);
+      DICTIONARY_CASE(INT16, Int16Type);
+      DICTIONARY_CASE(INT32, Int32Type);
+      DICTIONARY_CASE(INT64, Int64Type);
+      DICTIONARY_CASE(UINT8, UInt8Type);
+      DICTIONARY_CASE(UINT16, UInt16Type);
+      DICTIONARY_CASE(UINT32, UInt32Type);
+      DICTIONARY_CASE(UINT64, UInt64Type);
+      DICTIONARY_CASE(HALF_FLOAT, HalfFloatType);
+      DICTIONARY_CASE(FLOAT, FloatType);
+      DICTIONARY_CASE(DOUBLE, DoubleType);
+      DICTIONARY_CASE(DATE32, Date32Type);
+      DICTIONARY_CASE(DATE64, Date64Type);
+      DICTIONARY_CASE(BINARY, BinaryType);
+      DICTIONARY_CASE(STRING, StringType);
+      DICTIONARY_CASE(FIXED_SIZE_BINARY, FixedSizeBinaryType);
+      default:
+        return Status::NotImplemented("DictionaryArray converter for type ", t.ToString(),
+                                      " not implemented");
+    }
+  }
+
+  Status Visit(const StructType& t) {
+    using ConverterType = typename Converter::Struct;
+
+    std::shared_ptr<Converter> child_converter;
+    std::vector<std::shared_ptr<Converter>> child_converters;
+    std::vector<std::shared_ptr<ArrayBuilder>> child_builders;
+
+    for (const auto& field : t.fields()) {
+      ARROW_ASSIGN_OR_RAISE(child_converter,
+                            Converter::Make(field->type(), pool, options));
+
+      // TODO: use move
+      child_converters.push_back(child_converter);
+      child_builders.push_back(child_converter->builder());

Review comment:
       ```suggestion
         child_builders.push_back(child_converter->builder());
         child_converters.push_back(std::move(child_converter));
   ```

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter>
+struct MakeConverterImpl;
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  template <typename T>
+  using Primitive = PrimitiveConverter<T, Self>;
+  template <typename T>
+  using List = ListConverter<T, Self>;
+  template <typename T>
+  using Dictionary = DictionaryConverter<T, Self>;
+  using Struct = StructConverter<Self>;
+
+  static Result<std::shared_ptr<Self>> Make(std::shared_ptr<DataType> type,
+                                            MemoryPool* pool, OptionsType options) {
+    std::shared_ptr<Self> out;
+    MakeConverterImpl<Self> visitor = {type, pool, options, &out};
+    ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+    ARROW_RETURN_NOT_OK(out->Init());
+    return out;
+  }
+
+  virtual ~Converter() = default;
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>> children() const { return children_; }
+
+  virtual Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  virtual Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  friend struct MakeConverterImpl<Self>;
+
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+#define DICTIONARY_CASE(TYPE_ENUM, TYPE_CLASS)                          \
+  case Type::TYPE_ENUM:                                                 \
+    return Finish<typename Converter::template Dictionary<TYPE_CLASS>>( \
+        std::move(builder), {});                                        \
+    break;
+
+template <typename Converter>
+struct MakeConverterImpl {
+  Status Visit(const NullType& t) {
+    using BuilderType = typename TypeTraits<NullType>::BuilderType;
+    using ConverterType = typename Converter::template Primitive<NullType>;
+
+    auto builder = std::make_shared<BuilderType>(pool);
+    return Finish<ConverterType>(std::move(builder), {});
+  }
+
+  template <typename T>
+  enable_if_t<!is_nested_type<T>::value && !is_interval_type<T>::value &&
+                  !is_dictionary_type<T>::value && !is_extension_type<T>::value,
+              Status>
+  Visit(const T& t) {
+    using BuilderType = typename TypeTraits<T>::BuilderType;
+    using ConverterType = typename Converter::template Primitive<T>;
+
+    auto builder = std::make_shared<BuilderType>(type, pool);
+    return Finish<ConverterType>(std::move(builder), {});
+  }
+
+  template <typename T>
+  enable_if_t<is_list_like_type<T>::value && !std::is_same<T, MapType>::value, Status>
+  Visit(const T& t) {
+    using BuilderType = typename TypeTraits<T>::BuilderType;
+    using ConverterType = typename Converter::template List<T>;
+
+    ARROW_ASSIGN_OR_RAISE(auto child_converter,
+                          Converter::Make(t.value_type(), pool, options));
+    auto builder = std::make_shared<BuilderType>(pool, child_converter->builder(), type);
+    return Finish<ConverterType>(std::move(builder), {std::move(child_converter)});
+  }
+
+  Status Visit(const MapType& t) {
+    using ConverterType = typename Converter::template List<MapType>;
+
+    // TODO(kszucs): seems like builders not respect field nullability
+    std::vector<std::shared_ptr<Field>> struct_fields{t.key_field(), t.item_field()};
+    auto struct_type = std::make_shared<StructType>(struct_fields);
+    ARROW_ASSIGN_OR_RAISE(auto struct_converter,
+                          Converter::Make(struct_type, pool, options));
+
+    auto struct_builder = struct_converter->builder();
+    auto key_builder = struct_builder->child_builder(0);
+    auto item_builder = struct_builder->child_builder(1);
+    auto builder = std::make_shared<MapBuilder>(pool, key_builder, item_builder, type);
+
+    return Finish<ConverterType>(std::move(builder), {std::move(struct_converter)});
+  }
+
+  Status Visit(const DictionaryType& t) {
+    std::unique_ptr<ArrayBuilder> builder;
+    ARROW_RETURN_NOT_OK(MakeDictionaryBuilder(pool, type, NULLPTR, &builder));
+
+    switch (t.value_type()->id()) {
+      DICTIONARY_CASE(BOOL, BooleanType);
+      DICTIONARY_CASE(INT8, Int8Type);
+      DICTIONARY_CASE(INT16, Int16Type);
+      DICTIONARY_CASE(INT32, Int32Type);
+      DICTIONARY_CASE(INT64, Int64Type);
+      DICTIONARY_CASE(UINT8, UInt8Type);
+      DICTIONARY_CASE(UINT16, UInt16Type);
+      DICTIONARY_CASE(UINT32, UInt32Type);
+      DICTIONARY_CASE(UINT64, UInt64Type);
+      DICTIONARY_CASE(HALF_FLOAT, HalfFloatType);
+      DICTIONARY_CASE(FLOAT, FloatType);
+      DICTIONARY_CASE(DOUBLE, DoubleType);
+      DICTIONARY_CASE(DATE32, Date32Type);
+      DICTIONARY_CASE(DATE64, Date64Type);
+      DICTIONARY_CASE(BINARY, BinaryType);
+      DICTIONARY_CASE(STRING, StringType);
+      DICTIONARY_CASE(FIXED_SIZE_BINARY, FixedSizeBinaryType);
+      default:
+        return Status::NotImplemented("DictionaryArray converter for type ", t.ToString(),
+                                      " not implemented");
+    }
+  }
+
+  Status Visit(const StructType& t) {
+    using ConverterType = typename Converter::Struct;
+
+    std::shared_ptr<Converter> child_converter;
+    std::vector<std::shared_ptr<Converter>> child_converters;
+    std::vector<std::shared_ptr<ArrayBuilder>> child_builders;
+
+    for (const auto& field : t.fields()) {
+      ARROW_ASSIGN_OR_RAISE(child_converter,
+                            Converter::Make(field->type(), pool, options));
+
+      // TODO: use move
+      child_converters.push_back(child_converter);
+      child_builders.push_back(child_converter->builder());
+    }
+
+    auto builder = std::make_shared<StructBuilder>(type, pool, child_builders);

Review comment:
       ```suggestion
       auto builder = std::make_shared<StructBuilder>(std::move(type), pool, std::move(child_builders));
   ```

##########
File path: python/pyarrow/includes/libarrow.pxd
##########
@@ -969,11 +969,13 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
         vector[shared_ptr[CScalar]] value
         CResult[shared_ptr[CScalar]] field(CFieldRef ref) const
 
-    cdef cppclass CDictionaryScalar" arrow::DictionaryScalar"(CScalar):
-        cppclass CDictionaryValue "arrow::DictionaryScalar::ValueType":
-            shared_ptr[CScalar] index
-            shared_ptr[CArray] dictionary
+    cdef cppclass CDictionaryValue "arrow::DictionaryScalar::ValueType":
+        shared_ptr[CScalar] index
+        shared_ptr[CArray] dictionary
 
+    cdef cppclass CDictionaryScalar" arrow::DictionaryScalar"(CScalar):
+        CDictionaryScalar(CDictionaryValue value, shared_ptr[CDataType],
+                          c_bool is_valid)
         CDictionaryValue value

Review comment:
       Nit: make this class more clearly named
   ```suggestion
       cdef cppclass CDictionaryScalarIndexAndDictionary" arrow::DictionaryScalar::ValueType":
           shared_ptr[CScalar] index
           shared_ptr[CArray] dictionary
   
       cdef cppclass CDictionaryScalar" arrow::DictionaryScalar"(CScalar):
           CDictionaryScalar(CDictionaryScalarIndexAndDictionary value, shared_ptr[CDataType],
                             c_bool is_valid)
           CDictionaryScalarIndexAndDictionary value
   ```

##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -1352,64 +927,40 @@ Status ConvertToSequenceAndInferSize(PyObject* obj, PyObject** seq, int64_t* siz
   return Status::OK();
 }
 
-Status ConvertPySequence(PyObject* sequence_source, PyObject* mask,
-                         const PyConversionOptions& options,
-                         std::shared_ptr<ChunkedArray>* out) {
+Result<std::shared_ptr<ChunkedArray>> ConvertPySequence(PyObject* obj, PyObject* mask,
+                                                        const PyConversionOptions& opts,
+                                                        MemoryPool* pool) {
   PyAcquireGIL lock;
 
   PyObject* seq;
   OwnedRef tmp_seq_nanny;
-
-  std::shared_ptr<DataType> real_type;
+  PyConversionOptions options = opts;  // copy options struct since we modify it below

Review comment:
       For the same effect you could instead change the argument from `const PyConversionOptions&` to `PyConversionOptions`

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter>
+struct MakeConverterImpl;
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  template <typename T>
+  using Primitive = PrimitiveConverter<T, Self>;
+  template <typename T>
+  using List = ListConverter<T, Self>;
+  template <typename T>
+  using Dictionary = DictionaryConverter<T, Self>;
+  using Struct = StructConverter<Self>;
+
+  static Result<std::shared_ptr<Self>> Make(std::shared_ptr<DataType> type,
+                                            MemoryPool* pool, OptionsType options) {
+    std::shared_ptr<Self> out;
+    MakeConverterImpl<Self> visitor = {type, pool, options, &out};
+    ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+    ARROW_RETURN_NOT_OK(out->Init());
+    return out;
+  }
+
+  virtual ~Converter() = default;
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>> children() const { return children_; }
+
+  virtual Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  virtual Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  friend struct MakeConverterImpl<Self>;
+
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+#define DICTIONARY_CASE(TYPE_ENUM, TYPE_CLASS)                          \
+  case Type::TYPE_ENUM:                                                 \
+    return Finish<typename Converter::template Dictionary<TYPE_CLASS>>( \
+        std::move(builder), {});                                        \
+    break;
+
+template <typename Converter>
+struct MakeConverterImpl {
+  Status Visit(const NullType& t) {
+    using BuilderType = typename TypeTraits<NullType>::BuilderType;
+    using ConverterType = typename Converter::template Primitive<NullType>;

Review comment:
       Deriving converter types like this requires `PyConverter` and other implementations to be both intermediate implementations of specializations of `Converter<>` and traits mapping `DataType` subclasses to concrete converters. These responsibilities are orthogonal and users should not be forced to handle both in the same monolithic class.
   
   Furthermore, the mapping (`DataType` subclasses -> concrete converter) is unnecessarily convoluted: to get the `ConverterType` for `NullType` we use `typename Converter::template Primitive<NullType>` whereas to get the same for `MapType` we use `typename Converter::template List<MapType>`. This mapping should be contained in a single trait which applies to all `DataType` subclasses and can be specialized in however is most ergonomic at the point of extension. For example:
   ```c++
   template <typename DATA_TYPE>
   struct PyConverterType;
   
   template <>
   struct PyConverterType<NullType> {
     using type = PyNullConverter;
   };
   
   // Used in construction via:
   MakeConverter<PyConverterType>(...);
   ```

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter>
+struct MakeConverterImpl;
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  template <typename T>
+  using Primitive = PrimitiveConverter<T, Self>;
+  template <typename T>
+  using List = ListConverter<T, Self>;
+  template <typename T>
+  using Dictionary = DictionaryConverter<T, Self>;
+  using Struct = StructConverter<Self>;
+
+  static Result<std::shared_ptr<Self>> Make(std::shared_ptr<DataType> type,
+                                            MemoryPool* pool, OptionsType options) {
+    std::shared_ptr<Self> out;
+    MakeConverterImpl<Self> visitor = {type, pool, options, &out};
+    ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+    ARROW_RETURN_NOT_OK(out->Init());
+    return out;
+  }
+
+  virtual ~Converter() = default;
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>> children() const { return children_; }
+
+  virtual Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  virtual Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  friend struct MakeConverterImpl<Self>;
+
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+#define DICTIONARY_CASE(TYPE_ENUM, TYPE_CLASS)                          \
+  case Type::TYPE_ENUM:                                                 \
+    return Finish<typename Converter::template Dictionary<TYPE_CLASS>>( \
+        std::move(builder), {});                                        \
+    break;
+
+template <typename Converter>
+struct MakeConverterImpl {
+  Status Visit(const NullType& t) {
+    using BuilderType = typename TypeTraits<NullType>::BuilderType;
+    using ConverterType = typename Converter::template Primitive<NullType>;
+
+    auto builder = std::make_shared<BuilderType>(pool);
+    return Finish<ConverterType>(std::move(builder), {});
+  }
+
+  template <typename T>
+  enable_if_t<!is_nested_type<T>::value && !is_interval_type<T>::value &&
+                  !is_dictionary_type<T>::value && !is_extension_type<T>::value,
+              Status>
+  Visit(const T& t) {
+    using BuilderType = typename TypeTraits<T>::BuilderType;
+    using ConverterType = typename Converter::template Primitive<T>;
+
+    auto builder = std::make_shared<BuilderType>(type, pool);
+    return Finish<ConverterType>(std::move(builder), {});
+  }
+
+  template <typename T>
+  enable_if_t<is_list_like_type<T>::value && !std::is_same<T, MapType>::value, Status>
+  Visit(const T& t) {
+    using BuilderType = typename TypeTraits<T>::BuilderType;
+    using ConverterType = typename Converter::template List<T>;
+
+    ARROW_ASSIGN_OR_RAISE(auto child_converter,
+                          Converter::Make(t.value_type(), pool, options));
+    auto builder = std::make_shared<BuilderType>(pool, child_converter->builder(), type);
+    return Finish<ConverterType>(std::move(builder), {std::move(child_converter)});
+  }
+
+  Status Visit(const MapType& t) {
+    using ConverterType = typename Converter::template List<MapType>;
+
+    // TODO(kszucs): seems like builders not respect field nullability
+    std::vector<std::shared_ptr<Field>> struct_fields{t.key_field(), t.item_field()};
+    auto struct_type = std::make_shared<StructType>(struct_fields);
+    ARROW_ASSIGN_OR_RAISE(auto struct_converter,
+                          Converter::Make(struct_type, pool, options));
+
+    auto struct_builder = struct_converter->builder();
+    auto key_builder = struct_builder->child_builder(0);
+    auto item_builder = struct_builder->child_builder(1);
+    auto builder = std::make_shared<MapBuilder>(pool, key_builder, item_builder, type);
+
+    return Finish<ConverterType>(std::move(builder), {std::move(struct_converter)});
+  }
+
+  Status Visit(const DictionaryType& t) {
+    std::unique_ptr<ArrayBuilder> builder;
+    ARROW_RETURN_NOT_OK(MakeDictionaryBuilder(pool, type, NULLPTR, &builder));
+
+    switch (t.value_type()->id()) {
+      DICTIONARY_CASE(BOOL, BooleanType);
+      DICTIONARY_CASE(INT8, Int8Type);
+      DICTIONARY_CASE(INT16, Int16Type);
+      DICTIONARY_CASE(INT32, Int32Type);
+      DICTIONARY_CASE(INT64, Int64Type);
+      DICTIONARY_CASE(UINT8, UInt8Type);
+      DICTIONARY_CASE(UINT16, UInt16Type);
+      DICTIONARY_CASE(UINT32, UInt32Type);
+      DICTIONARY_CASE(UINT64, UInt64Type);
+      DICTIONARY_CASE(HALF_FLOAT, HalfFloatType);
+      DICTIONARY_CASE(FLOAT, FloatType);
+      DICTIONARY_CASE(DOUBLE, DoubleType);
+      DICTIONARY_CASE(DATE32, Date32Type);
+      DICTIONARY_CASE(DATE64, Date64Type);
+      DICTIONARY_CASE(BINARY, BinaryType);
+      DICTIONARY_CASE(STRING, StringType);
+      DICTIONARY_CASE(FIXED_SIZE_BINARY, FixedSizeBinaryType);
+      default:
+        return Status::NotImplemented("DictionaryArray converter for type ", t.ToString(),
+                                      " not implemented");
+    }
+  }
+
+  Status Visit(const StructType& t) {
+    using ConverterType = typename Converter::Struct;
+
+    std::shared_ptr<Converter> child_converter;
+    std::vector<std::shared_ptr<Converter>> child_converters;
+    std::vector<std::shared_ptr<ArrayBuilder>> child_builders;
+
+    for (const auto& field : t.fields()) {
+      ARROW_ASSIGN_OR_RAISE(child_converter,
+                            Converter::Make(field->type(), pool, options));
+
+      // TODO: use move
+      child_converters.push_back(child_converter);
+      child_builders.push_back(child_converter->builder());
+    }
+
+    auto builder = std::make_shared<StructBuilder>(type, pool, child_builders);
+    return Finish<ConverterType>(std::move(builder), std::move(child_converters));
+  }
+
+  Status Visit(const DataType& t) { return Status::NotImplemented(t.name()); }
+
+  template <typename ConverterType>
+  Status Finish(std::shared_ptr<ArrayBuilder> builder,
+                std::vector<std::shared_ptr<Converter>> children) {
+    auto converter = new ConverterType();
+    converter->type_ = std::move(type);
+    converter->builder_ = std::move(builder);
+    converter->options_ = options;
+    converter->children_ = std::move(children);
+    out->reset(converter);
+    return Status::OK();
+  }
+
+  const std::shared_ptr<DataType> type;
+  MemoryPool* pool;
+  typename Converter::OptionsType options;
+  std::shared_ptr<Converter>* out;
+};
+
+// TODO(kszucs): rename to AutoChunker
+template <typename BaseConverter>
+class Chunker : public BaseConverter {

Review comment:
       I don't think Chunker should inherit BaseConverter. Chunker interacts with a BaseConverter via composition, so inheritance is redundant and leads to confusion about which BaseConverter instance is being used. For example, `Chunker::Reserve()` calls Reserve on Chunker::builder_ but *not* on Chunker::converter_, so any overrides of `BaseConverter::Reserve` will be ignored.

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter>
+struct MakeConverterImpl;
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  template <typename T>
+  using Primitive = PrimitiveConverter<T, Self>;
+  template <typename T>
+  using List = ListConverter<T, Self>;
+  template <typename T>
+  using Dictionary = DictionaryConverter<T, Self>;
+  using Struct = StructConverter<Self>;
+
+  static Result<std::shared_ptr<Self>> Make(std::shared_ptr<DataType> type,
+                                            MemoryPool* pool, OptionsType options) {
+    std::shared_ptr<Self> out;
+    MakeConverterImpl<Self> visitor = {type, pool, options, &out};
+    ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+    ARROW_RETURN_NOT_OK(out->Init());
+    return out;
+  }
+
+  virtual ~Converter() = default;
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>> children() const { return children_; }
+
+  virtual Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  virtual Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }

Review comment:
       Same

##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -329,186 +300,106 @@ struct ValueConverter<DurationType> {
         default:
           return Status::UnknownError("Invalid time unit");
       }
+    } else if (PyArray_CheckAnyScalarExact(obj)) {
+      // validate that the numpy scalar has np.datetime64 dtype
+      std::shared_ptr<DataType> numpy_type;
+      RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &numpy_type));
+      if (!numpy_type->Equals(*type)) {
+        return Status::NotImplemented("Expected np.timedelta64 but got: ",
+                                      numpy_type->ToString());
+      }
+      return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
     } else {
       RETURN_NOT_OK(internal::CIntFromPython(obj, &value));
     }
     return value;
   }
 
-  static inline Result<int64_t> FromNumpy(PyObject* obj, TimeUnit::type unit) {
-    // validate that the numpy scalar has np.timedelta64 dtype
-    std::shared_ptr<DataType> type;
-    RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &type));
-    if (type->id() != DurationType::type_id) {
-      // TODO(kszucs): the message should highlight the received numpy dtype
-      return Status::Invalid("Expected np.timedelta64 but got: ", type->ToString());
-    }
-    // validate that the time units are matching
-    if (unit != checked_cast<const DurationType&>(*type).unit()) {
-      return Status::NotImplemented(
-          "Cannot convert NumPy np.timedelta64 objects with differing unit");
-    }
-    // convert the numpy value
-    return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_any_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    return std::move(view);
-  }
-};
+  // The binary-like intermediate representation is PyBytesView because it keeps temporary
+  // python objects alive (non-contiguous memoryview) and stores whether the original
+  // object was unicode encoded or not, which is used for unicode -> bytes coersion if
+  // there is a non-unicode object observed.
 
-template <typename Type>
-struct ValueConverter<Type, enable_if_string_like<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    // strict conversion, force output to be unicode / utf8 and validate that
-    // any binary values are utf8
-    bool is_utf8 = false;
-    PyBytesView view;
-
-    RETURN_NOT_OK(view.FromString(obj, &is_utf8));
-    if (!is_utf8) {
-      return internal::InvalidValue(obj, "was not a utf8 string");
-    }
-    return std::move(view);
+  static Result<PyBytesView> Convert(const BaseBinaryType*, const O&, I obj) {
+    return PyBytesView::FromString(obj);
   }
 
-  static inline Result<PyBytesView> FromPython(PyObject* obj, bool* is_utf8) {
-    PyBytesView view;
-
-    // Non-strict conversion; keep track of whether values are unicode or bytes
-    if (PyUnicode_Check(obj)) {
-      *is_utf8 = true;
-      RETURN_NOT_OK(view.FromUnicode(obj));
+  static Result<PyBytesView> Convert(const FixedSizeBinaryType* type, const O&, I obj) {
+    ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj));
+    if (ARROW_PREDICT_TRUE(view.size == type->byte_width())) {
+      return std::move(view);
     } else {
-      // If not unicode or bytes, FromBinary will error
-      *is_utf8 = false;
-      RETURN_NOT_OK(view.FromBinary(obj));
-    }
-    return std::move(view);
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_fixed_size_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj, int32_t byte_width) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    if (ARROW_PREDICT_FALSE(view.size != byte_width)) {
       std::stringstream ss;
-      ss << "expected to be length " << byte_width << " was " << view.size;
+      ss << "expected to be length " << type->byte_width() << " was " << view.size;
       return internal::InvalidValue(obj, ss.str());
-    } else {
-      return std::move(view);
     }
   }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter base and CRTP "middle" subclasses
 
-class SeqConverter;
-
-// Forward-declare converter factory
-Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
-                    bool strict_conversions, bool ignore_timezone,
-                    std::unique_ptr<SeqConverter>* out);
-
-// Marshal Python sequence (list, tuple, etc.) to Arrow array
-class SeqConverter {
- public:
-  virtual ~SeqConverter() = default;
-
-  // Initialize the sequence converter with an ArrayBuilder created
-  // externally. The reason for this interface is that we have
-  // arrow::MakeBuilder which also creates child builders for nested types, so
-  // we have to pass in the child builders to child SeqConverter in the case of
-  // converting Python objects to Arrow nested types
-  virtual Status Init(ArrayBuilder* builder) = 0;
-
-  // Append a single null value to the builder
-  virtual Status AppendNull() = 0;
-
-  // Append a valid value
-  virtual Status AppendValue(PyObject* seq) = 0;
-
-  // Append a single python object handling Null values
-  virtual Status Append(PyObject* seq) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status Extend(PyObject* seq, int64_t size) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status ExtendMasked(PyObject* seq, PyObject* mask, int64_t size) = 0;
-
-  virtual Status Close() {
-    if (chunks_.size() == 0 || builder_->length() > 0) {
-      std::shared_ptr<Array> last_chunk;
-      RETURN_NOT_OK(builder_->Finish(&last_chunk));
-      chunks_.emplace_back(std::move(last_chunk));
+  template <typename T>
+  static enable_if_string<T, Result<PyBytesView>> Convert(const T*, const O& options,
+                                                          I obj) {
+    if (options.strict) {
+      // Strict conversion, force output to be unicode / utf8 and validate that
+      // any binary values are utf8
+      ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj, true));
+      if (!view.is_utf8) {
+        return internal::InvalidValue(obj, "was not a utf8 string");
+      }
+      return std::move(view);
+    } else {
+      // Non-strict conversion; keep track of whether values are unicode or bytes
+      return PyBytesView::FromString(obj);
     }
-    return Status::OK();
   }
 
-  virtual Status GetResult(std::shared_ptr<ChunkedArray>* out) {
-    // Still some accumulated data in the builder. If there are no chunks, we
-    // always call Finish to deal with the edge case where a size-0 sequence
-    // was converted with a specific output type, like array([], type=t)
-    RETURN_NOT_OK(Close());
-    *out = std::make_shared<ChunkedArray>(this->chunks_, builder_->type());
-    return Status::OK();
+  static Result<bool> Convert(const DataType* type, const O&, I obj) {
+    return Status::NotImplemented("PyValue::Convert is not implemented for type ", type);
   }
-
-  ArrayBuilder* builder() const { return builder_; }
-
-  int num_chunks() const { return static_cast<int>(chunks_.size()); }
-
- protected:
-  ArrayBuilder* builder_;
-  bool unfinished_builder_;
-  std::vector<std::shared_ptr<Array>> chunks_;
 };
 
-template <typename Type, NullCoding null_coding = NullCoding::NONE_ONLY>
-class TypedConverter : public SeqConverter {
- public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
+// Forward-declare the type-family specific converters to inject them to the PyConverter
+// base class as type aliases.
 
-  Status Init(ArrayBuilder* builder) override {
-    builder_ = builder;
-    DCHECK_NE(builder_, nullptr);
-    typed_builder_ = checked_cast<BuilderType*>(builder);
-    return Status::OK();
-  }
+template <typename T, typename Enable = void>
+class PyPrimitiveConverter;
 
-  // Append a missing item (default implementation)
-  Status AppendNull() override { return this->typed_builder_->AppendNull(); }
+template <typename T, typename Enable = void>
+class PyDictionaryConverter;
 
-  // Append null if the obj is None or pandas null otherwise the valid value
-  Status Append(PyObject* obj) override {
-    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
-  }
+template <typename T>
+class PyListConverter;
 
-  Status Extend(PyObject* obj, int64_t size) override {
+class PyStructConverter;
+
+// The base Converter class is a mixin with predefined behavior and constructors.
+class PyConverter : public Converter<PyObject*, PyConversionOptions, PyConverter> {
+ public:
+  // Type aliases used by the parent Converter mixin's factory.
+  template <typename T>
+  using Primitive = PyPrimitiveConverter<T>;
+  template <typename T>
+  using Dictionary = PyDictionaryConverter<T>;
+  template <typename T>
+  using List = PyListConverter<T>;
+  using Struct = PyStructConverter;
+
+  // Convert and append a sequence of values
+  Status Extend(PyObject* values, int64_t size) {

Review comment:
       I think making this (and ExtendMasked) a member function is adding to the complexity of this heirarchy unnecessarily; since it only uses public methods there is no reason not to extract it as a free function:
   
   ```c++
   Status Extend(PyConverter* converter, PyObject* values, int64_t size) {
     /// Ensure we've allocated enough space 
     RETURN_NOT_OK(converter->Reserve(size));
     // Iterate over the items adding each one
     return internal::VisitSequence(values, [converter](PyObject* item, bool* /*unused*/) {
       return converter->Append(item);
     });
   }
   ```

##########
File path: python/pyarrow/array.pxi
##########
@@ -158,24 +158,44 @@ def array(object obj, type=None, mask=None, size=None, from_pandas=None,
     Notes
     -----
     Localized timestamps will currently be returned as UTC (pandas's native
-    representation).  Timezone-naive data will be implicitly interpreted as
+    representation). Timezone-naive data will be implicitly interpreted as
     UTC.
 
+    Converting to dictionary array will choose to use a larger integer type for
+    the indices if the number of distict values wouldn't fit to the range of
+    the passed type. This adaptive nature means that if there are more than 127
+    values the returned dictionary array's key type is going to be pa.int16()
+    even if pa.int8() was passed to the function. Note that smaller key type
+    than the passed one won't be chosed.

Review comment:
       ```suggestion
       Converting to dictionary array will promote to a wider integer type for
       indices if the number of distinct values cannot be represented, even if
       the index type was explicitly set. This means that if there are more than 127
       values the returned dictionary array's index type will be at least pa.int16()
       even if pa.int8() was passed to the function. Note that an explicit index type
       will not be demoted even if it is wider than required.
   ```

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;

Review comment:
       ```suggestion
   ```
   
   These are not necessary here since we're in the namespace `arrow::internal::` already

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter>
+struct MakeConverterImpl;
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  template <typename T>
+  using Primitive = PrimitiveConverter<T, Self>;
+  template <typename T>
+  using List = ListConverter<T, Self>;
+  template <typename T>
+  using Dictionary = DictionaryConverter<T, Self>;
+  using Struct = StructConverter<Self>;
+
+  static Result<std::shared_ptr<Self>> Make(std::shared_ptr<DataType> type,
+                                            MemoryPool* pool, OptionsType options) {
+    std::shared_ptr<Self> out;
+    MakeConverterImpl<Self> visitor = {type, pool, options, &out};
+    ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+    ARROW_RETURN_NOT_OK(out->Init());
+    return out;
+  }
+
+  virtual ~Converter() = default;
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>> children() const { return children_; }
+
+  virtual Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  virtual Status AppendNull() { return builder_->AppendNull(); }

Review comment:
       I don't think this needs to be virtual if Chunker no longer inherits BaseConverter; ArrayBuilder::AppendNull is already virtual and ensures all descendant builders are appended to correctly.




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

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



[GitHub] [arrow] bkietz commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,296 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+template <typename BaseConverter, template <typename...> class ConverterTrait>
+static Result<std::shared_ptr<BaseConverter>> MakeConverter(
+    std::shared_ptr<DataType> type, typename BaseConverter::OptionsType options,
+    MemoryPool* pool);
+
+template <typename Input, typename Options>
+class Converter {
+ public:
+  using Self = Converter<Input, Options>;
+  using InputType = Input;
+  using OptionsType = Options;
+
+  virtual ~Converter() = default;
+
+  Status Construct(std::shared_ptr<DataType> type, OptionsType options,
+                   MemoryPool* pool) {
+    type_ = std::move(type);
+    options_ = std::move(options);
+    return Init(pool);
+  }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }
+
+  const std::shared_ptr<ArrayBuilder>& builder() const { return builder_; }
+
+  const std::shared_ptr<DataType>& type() const { return type_; }
+
+  OptionsType options() const { return options_; }
+
+  const std::vector<std::shared_ptr<Self>>& children() const { return children_; }
+
+  Status Reserve(int64_t additional_capacity) {
+    return builder_->Reserve(additional_capacity);
+  }
+
+  Status AppendNull() { return builder_->AppendNull(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray() { return builder_->Finish(); }
+
+  virtual Result<std::shared_ptr<Array>> ToArray(int64_t length) {
+    ARROW_ASSIGN_OR_RAISE(auto arr, this->ToArray());
+    return arr->Slice(0, length);
+  }
+
+ protected:
+  virtual Status Init(MemoryPool* pool) { return Status::OK(); }
+
+  std::shared_ptr<DataType> type_;
+  std::shared_ptr<ArrayBuilder> builder_;
+  std::vector<std::shared_ptr<Self>> children_;
+  OptionsType options_;
+};
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+ protected:
+  Status Init(MemoryPool* pool) override {
+    this->builder_ = std::make_shared<BuilderType>(this->type_, pool);
+    this->primitive_type_ = checked_cast<const T*>(this->type_.get());
+    this->primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter, template <typename...> class ConverterTrait>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+  using ConverterType = typename ConverterTrait<T>::type;
+
+ protected:
+  Status Init(MemoryPool* pool) override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    ARROW_ASSIGN_OR_RAISE(value_converter_,
+                          (MakeConverter<BaseConverter, ConverterTrait>(
+                              list_type_->value_type(), this->options_, pool)));
+    this->builder_ =
+        std::make_shared<BuilderType>(pool, value_converter_->builder(), this->type_);
+    this->children_ = {value_converter_};
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter, template <typename...> class ConverterTrait>
+class StructConverter : public BaseConverter {
+ public:
+  using ConverterType = typename ConverterTrait<StructType>::type;
+
+ protected:
+  Status Init(MemoryPool* pool) override {
+    std::shared_ptr<BaseConverter> child_converter;
+    std::vector<std::shared_ptr<ArrayBuilder>> child_builders;
+
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    for (const auto& field : struct_type_->fields()) {
+      ARROW_ASSIGN_OR_RAISE(child_converter,
+                            (MakeConverter<BaseConverter, ConverterTrait>(
+                                field->type(), this->options_, pool)));
+      child_builders.push_back(child_converter->builder());
+      this->children_.push_back(std::move(child_converter));
+    }
+
+    this->builder_ =
+        std::make_shared<StructBuilder>(this->type_, pool, std::move(child_builders));
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+
+    return Status::OK();
+  }
+
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+ protected:
+  Status Init(MemoryPool* pool) override {
+    std::unique_ptr<ArrayBuilder> builder;
+    ARROW_RETURN_NOT_OK(MakeDictionaryBuilder(pool, this->type_, NULLPTR, &builder));
+    this->builder_ = std::move(builder);
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename BaseConverter, template <typename...> class ConverterTrait>
+struct MakeConverterImpl {
+  template <typename T, typename ConverterType = typename ConverterTrait<T>::type>
+  Status Visit(const T&) {
+    out.reset(new ConverterType());
+    return out->Construct(std::move(type), std::move(options), pool);
+  }
+
+  Status Visit(const DictionaryType& t) {
+    switch (t.value_type()->id()) {
+#define DICTIONARY_CASE(TYPE)                                            \
+  case TYPE::type_id:                                                    \
+    out = std::make_shared<                                              \
+        typename ConverterTrait<DictionaryType>::template type<TYPE>>(); \
+    break;
+      DICTIONARY_CASE(BooleanType);
+      DICTIONARY_CASE(Int8Type);
+      DICTIONARY_CASE(Int16Type);
+      DICTIONARY_CASE(Int32Type);
+      DICTIONARY_CASE(Int64Type);
+      DICTIONARY_CASE(UInt8Type);
+      DICTIONARY_CASE(UInt16Type);
+      DICTIONARY_CASE(UInt32Type);
+      DICTIONARY_CASE(UInt64Type);
+      DICTIONARY_CASE(FloatType);
+      DICTIONARY_CASE(DoubleType);
+      DICTIONARY_CASE(BinaryType);
+      DICTIONARY_CASE(StringType);
+      DICTIONARY_CASE(FixedSizeBinaryType);
+      default:
+        return Status::NotImplemented("DictionaryArray converter for type ", t.ToString(),
+                                      " not implemented");
+    }
+    return out->Construct(std::move(type), std::move(options), pool);
+  }
+
+  Status Visit(const DataType& t) { return Status::NotImplemented(t.name()); }
+
+  std::shared_ptr<DataType> type;
+  typename BaseConverter::OptionsType options;
+  MemoryPool* pool;
+  std::shared_ptr<BaseConverter> out;
+};
+
+template <typename BaseConverter, template <typename...> class ConverterTrait>
+static Result<std::shared_ptr<BaseConverter>> MakeConverter(
+    std::shared_ptr<DataType> type, typename BaseConverter::OptionsType options,
+    MemoryPool* pool) {
+  MakeConverterImpl<BaseConverter, ConverterTrait> visitor{
+      std::move(type), std::move(options), pool, nullptr};

Review comment:
       ```suggestion
         std::move(type), std::move(options), pool, NULLPTR};
   ```

##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -329,985 +302,602 @@ struct ValueConverter<DurationType> {
         default:
           return Status::UnknownError("Invalid time unit");
       }
+    } else if (PyArray_CheckAnyScalarExact(obj)) {
+      // validate that the numpy scalar has np.datetime64 dtype
+      std::shared_ptr<DataType> numpy_type;
+      RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &numpy_type));
+      if (!numpy_type->Equals(*type)) {
+        return Status::NotImplemented("Expected np.timedelta64 but got: ",
+                                      numpy_type->ToString());
+      }
+      return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
     } else {
       RETURN_NOT_OK(internal::CIntFromPython(obj, &value));
     }
     return value;
   }
 
-  static inline Result<int64_t> FromNumpy(PyObject* obj, TimeUnit::type unit) {
-    // validate that the numpy scalar has np.timedelta64 dtype
-    std::shared_ptr<DataType> type;
-    RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &type));
-    if (type->id() != DurationType::type_id) {
-      // TODO(kszucs): the message should highlight the received numpy dtype
-      return Status::Invalid("Expected np.timedelta64 but got: ", type->ToString());
-    }
-    // validate that the time units are matching
-    if (unit != checked_cast<const DurationType&>(*type).unit()) {
-      return Status::NotImplemented(
-          "Cannot convert NumPy np.timedelta64 objects with differing unit");
-    }
-    // convert the numpy value
-    return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_any_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    return std::move(view);
-  }
-};
+  // The binary-like intermediate representation is PyBytesView because it keeps temporary
+  // python objects alive (non-contiguous memoryview) and stores whether the original
+  // object was unicode encoded or not, which is used for unicode -> bytes coersion if
+  // there is a non-unicode object observed.
 
-template <typename Type>
-struct ValueConverter<Type, enable_if_string_like<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    // strict conversion, force output to be unicode / utf8 and validate that
-    // any binary values are utf8
-    bool is_utf8 = false;
-    PyBytesView view;
-
-    RETURN_NOT_OK(view.FromString(obj, &is_utf8));
-    if (!is_utf8) {
-      return internal::InvalidValue(obj, "was not a utf8 string");
-    }
-    return std::move(view);
+  static Result<PyBytesView> Convert(const BaseBinaryType*, const O&, I obj) {
+    return PyBytesView::FromString(obj);
   }
 
-  static inline Result<PyBytesView> FromPython(PyObject* obj, bool* is_utf8) {
-    PyBytesView view;
-
-    // Non-strict conversion; keep track of whether values are unicode or bytes
-    if (PyUnicode_Check(obj)) {
-      *is_utf8 = true;
-      RETURN_NOT_OK(view.FromUnicode(obj));
+  static Result<PyBytesView> Convert(const FixedSizeBinaryType* type, const O&, I obj) {
+    ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj));
+    if (ARROW_PREDICT_TRUE(view.size == type->byte_width())) {
+      return std::move(view);
     } else {
-      // If not unicode or bytes, FromBinary will error
-      *is_utf8 = false;
-      RETURN_NOT_OK(view.FromBinary(obj));
-    }
-    return std::move(view);
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_fixed_size_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj, int32_t byte_width) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    if (ARROW_PREDICT_FALSE(view.size != byte_width)) {
       std::stringstream ss;
-      ss << "expected to be length " << byte_width << " was " << view.size;
+      ss << "expected to be length " << type->byte_width() << " was " << view.size;
       return internal::InvalidValue(obj, ss.str());
-    } else {
-      return std::move(view);
     }
   }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter base and CRTP "middle" subclasses
-
-class SeqConverter;
-
-// Forward-declare converter factory
-Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
-                    bool strict_conversions, bool ignore_timezone,
-                    std::unique_ptr<SeqConverter>* out);
-
-// Marshal Python sequence (list, tuple, etc.) to Arrow array
-class SeqConverter {
- public:
-  virtual ~SeqConverter() = default;
-
-  // Initialize the sequence converter with an ArrayBuilder created
-  // externally. The reason for this interface is that we have
-  // arrow::MakeBuilder which also creates child builders for nested types, so
-  // we have to pass in the child builders to child SeqConverter in the case of
-  // converting Python objects to Arrow nested types
-  virtual Status Init(ArrayBuilder* builder) = 0;
-
-  // Append a single null value to the builder
-  virtual Status AppendNull() = 0;
-
-  // Append a valid value
-  virtual Status AppendValue(PyObject* seq) = 0;
-
-  // Append a single python object handling Null values
-  virtual Status Append(PyObject* seq) = 0;
 
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status Extend(PyObject* seq, int64_t size) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status ExtendMasked(PyObject* seq, PyObject* mask, int64_t size) = 0;
-
-  virtual Status Close() {
-    if (chunks_.size() == 0 || builder_->length() > 0) {
-      std::shared_ptr<Array> last_chunk;
-      RETURN_NOT_OK(builder_->Finish(&last_chunk));
-      chunks_.emplace_back(std::move(last_chunk));
+  template <typename T>
+  static enable_if_string<T, Result<PyBytesView>> Convert(const T*, const O& options,
+                                                          I obj) {
+    if (options.strict) {
+      // Strict conversion, force output to be unicode / utf8 and validate that
+      // any binary values are utf8
+      ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj, true));
+      if (!view.is_utf8) {
+        return internal::InvalidValue(obj, "was not a utf8 string");
+      }
+      return std::move(view);
+    } else {
+      // Non-strict conversion; keep track of whether values are unicode or bytes
+      return PyBytesView::FromString(obj);
     }
-    return Status::OK();
   }
 
-  virtual Status GetResult(std::shared_ptr<ChunkedArray>* out) {
-    // Still some accumulated data in the builder. If there are no chunks, we
-    // always call Finish to deal with the edge case where a size-0 sequence
-    // was converted with a specific output type, like array([], type=t)
-    RETURN_NOT_OK(Close());
-    *out = std::make_shared<ChunkedArray>(this->chunks_, builder_->type());
-    return Status::OK();
+  static Result<bool> Convert(const DataType* type, const O&, I obj) {
+    return Status::NotImplemented("PyValue::Convert is not implemented for type ", type);
   }
-
-  ArrayBuilder* builder() const { return builder_; }
-
-  int num_chunks() const { return static_cast<int>(chunks_.size()); }
-
- protected:
-  ArrayBuilder* builder_;
-  bool unfinished_builder_;
-  std::vector<std::shared_ptr<Array>> chunks_;
 };
 
-template <typename Type, NullCoding null_coding = NullCoding::NONE_ONLY>
-class TypedConverter : public SeqConverter {
- public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  Status Init(ArrayBuilder* builder) override {
-    builder_ = builder;
-    DCHECK_NE(builder_, nullptr);
-    typed_builder_ = checked_cast<BuilderType*>(builder);
-    return Status::OK();
-  }
-
-  // Append a missing item (default implementation)
-  Status AppendNull() override { return this->typed_builder_->AppendNull(); }
-
-  // Append null if the obj is None or pandas null otherwise the valid value
-  Status Append(PyObject* obj) override {
-    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
-  }
-
-  Status Extend(PyObject* obj, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequence(
-        obj, [this](PyObject* item, bool* /* unused */) { return this->Append(item); });
-  }
-
-  Status ExtendMasked(PyObject* obj, PyObject* mask, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequenceMasked(
-        obj, mask, [this](PyObject* item, bool is_masked, bool* /* unused */) {
-          if (is_masked) {
-            return this->AppendNull();
-          } else {
-            // This will also apply the null-checking convention in the event
-            // that the value is not masked
-            return this->Append(item);  // perhaps use AppendValue instead?
-          }
-        });
-  }
+template <typename T>
+Status Extend(T* converter, PyObject* values, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequence(values, [converter](PyObject* item, bool* /* unused */) {
+    return converter->Append(item);
+  });
+}
 
- protected:
-  BuilderType* typed_builder_;
-};
+// Convert and append a sequence of values masked with a numpy array
+template <typename T>
+Status ExtendMasked(T* converter, PyObject* values, PyObject* mask, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequenceMasked(
+      values, mask, [converter](PyObject* item, bool is_masked, bool* /* unused */) {
+        if (is_masked) {
+          return converter->AppendNull();
+        } else {
+          // This will also apply the null-checking convention in the event
+          // that the value is not masked
+          return converter->Append(item);  // perhaps use AppendValue instead?
+        }
+      });
+}
 
-// ----------------------------------------------------------------------
-// Sequence converter for null type
+// The base Converter class is a mixin with predefined behavior and constructors.
+using PyConverter = Converter<PyObject*, PyConversionOptions>;
 
-template <NullCoding null_coding>
-class NullConverter : public TypedConverter<NullType, null_coding> {
- public:
-  Status AppendValue(PyObject* obj) override {
-    return internal::InvalidValue(obj, "converting to null type");
-  }
-};
+template <typename T, typename Enable = void>
+class PyPrimitiveConverter;
 
-// ----------------------------------------------------------------------
-// Sequence converter template for primitive (integer and floating point bool) types
+template <typename T>
+class PyListConverter;
 
-template <typename Type, NullCoding null_coding>
-class PrimitiveConverter : public TypedConverter<Type, null_coding> {
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value, ValueConverter<Type>::FromPython(obj));
-    return this->typed_builder_->Append(value);
-  }
-};
+template <typename U, typename Enable = void>
+class PyDictionaryConverter;
 
-// ----------------------------------------------------------------------
-// Sequence converters for temporal types
+class PyStructConverter;
 
-template <typename Type, NullCoding null_coding>
-class TimeConverter : public TypedConverter<Type, null_coding> {
- public:
-  explicit TimeConverter(TimeUnit::type unit, bool ignore_timezone)
-      : unit_(unit), ignore_timezone_(ignore_timezone) {}
-
-  // TODO(kszucs): support numpy values for date and time converters
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value,
-                          ValueConverter<Type>::FromPython(obj, unit_, ignore_timezone_));
-    return this->typed_builder_->Append(value);
-  }
+template <typename T, typename Enable = void>
+struct PyConverterTrait;
 
- protected:
-  TimeUnit::type unit_;
-  bool ignore_timezone_;
+template <typename T>
+struct PyConverterTrait<
+    T, enable_if_t<!is_nested_type<T>::value && !is_interval_type<T>::value &&
+                   !is_extension_type<T>::value>> {
+  using type = PyPrimitiveConverter<T>;
 };
 
-// TODO(kszucs): move it to the type_traits
 template <typename T>
-struct NumpyType {};
+struct PyConverterTrait<T, enable_if_list_like<T>> {
+  using type = PyListConverter<T>;
+};
 
 template <>
-struct NumpyType<TimestampType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_DATETIME>::isnull(v);
-  }
+struct PyConverterTrait<StructType> {
+  using type = PyStructConverter;
 };
 
 template <>
-struct NumpyType<DurationType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_TIMEDELTA>::isnull(v);
-  }
+struct PyConverterTrait<DictionaryType> {
+  template <typename T>
+  using type = PyDictionaryConverter<T>;
 };
 
-template <typename Type, NullCoding null_coding>
-class TemporalConverter : public TimeConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_null_type<T>::value || is_boolean_type<T>::value ||
+                   is_number_type<T>::value || is_decimal_type<T>::value ||
+                   is_date_type<T>::value || is_time_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using TimeConverter<Type, null_coding>::TimeConverter;
-
-  Status AppendValue(PyObject* obj) override {
-    int64_t value;
-    if (PyArray_CheckAnyScalarExact(obj)) {
-      // convert np.datetime64 / np.timedelta64 depending on Type
-      ARROW_ASSIGN_OR_RAISE(value, ValueConverter<Type>::FromNumpy(obj, this->unit_));
-      if (NumpyType<Type>::isnull(value)) {
-        // checks numpy NaT sentinel after conversion
-        return this->typed_builder_->AppendNull();
-      }
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
     } else {
       ARROW_ASSIGN_OR_RAISE(
-          value,
-          ValueConverter<Type>::FromPython(
-              obj, this->unit_, TimeConverter<Type, null_coding>::ignore_timezone_));
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      return this->primitive_builder_->Append(converted);
     }
-    return this->typed_builder_->Append(value);
   }
 };
 
-// ----------------------------------------------------------------------
-// Sequence converters for Binary, FixedSizeBinary, String
-
-template <typename Type, NullCoding null_coding>
-class BinaryLikeConverter : public TypedConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_timestamp_type<T>::value || is_duration_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  inline Status AutoChunk(Py_ssize_t size) {
-    // did we reach the builder size limit?
-    if (ARROW_PREDICT_FALSE(this->typed_builder_->value_data_length() + size >
-                            BuilderType::memory_limit())) {
-      // builder would be full, so need to add a new chunk
-      std::shared_ptr<Array> chunk;
-      RETURN_NOT_OK(this->typed_builder_->Finish(&chunk));
-      this->chunks_.emplace_back(std::move(chunk));
-    }
-    return Status::OK();
-  }
-
-  Status AppendString(const PyBytesView& view) {
-    // check that the value fits in the datatype
-    if (view.size > BuilderType::memory_limit()) {
-      return Status::Invalid("string too large for datatype");
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      // Numpy NaT sentinels can be checked after the conversion
+      if (PyArray_CheckAnyScalarExact(value) &&
+          PyValue::IsNaT(this->primitive_type_, converted)) {
+        return this->primitive_builder_->AppendNull();
+      } else {
+        return this->primitive_builder_->Append(converted);
+      }
     }
-    DCHECK_GE(view.size, 0);
-
-    // create a new chunk if the value would overflow the builder
-    RETURN_NOT_OK(AutoChunk(view.size));
-
-    // now we can safely append the value to the builder
-    RETURN_NOT_OK(
-        this->typed_builder_->Append(::arrow::util::string_view(view.bytes, view.size)));
-
-    return Status::OK();
   }
-
- protected:
-  // Create a single instance of PyBytesView here to prevent unnecessary object
-  // creation/destruction
-  PyBytesView string_view_;
 };
 
-template <typename Type, NullCoding null_coding>
-class BinaryConverter : public BinaryLikeConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_binary<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto view, ValueConverter<Type>::FromPython(obj));
-    return this->AppendString(view);
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
+    }
   }
 };
 
-template <NullCoding null_coding>
-class FixedSizeBinaryConverter
-    : public BinaryLikeConverter<FixedSizeBinaryType, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_string_like<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  explicit FixedSizeBinaryConverter(int32_t byte_width) : byte_width_(byte_width) {}
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      if (!view.is_utf8) {
+        // observed binary value
+        observed_binary_ = true;
+      }
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
+    }
+  }
 
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(
-        this->string_view_,
-        ValueConverter<FixedSizeBinaryType>::FromPython(obj, byte_width_));
-    return this->AppendString(this->string_view_);
+  Result<std::shared_ptr<Array>> ToArray() override {
+    ARROW_ASSIGN_OR_RAISE(auto array, (PrimitiveConverter<T, PyConverter>::ToArray()));
+    if (observed_binary_) {
+      // if we saw any non-unicode, cast results to BinaryArray
+      auto binary_type = TypeTraits<typename T::PhysicalType>::type_singleton();
+      return array->View(binary_type);
+    } else {
+      return array;
+    }
   }
 
  protected:
-  int32_t byte_width_;
+  bool observed_binary_ = false;
 };
 
-// For String/UTF8, if strict_conversions enabled, we reject any non-UTF8,
-// otherwise we allow but return results as BinaryArray
-template <typename Type, bool Strict, NullCoding null_coding>
-class StringConverter : public BinaryLikeConverter<Type, null_coding> {
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_c_type<U>>
+    : public DictionaryConverter<U, PyConverter> {
  public:
-  StringConverter() : binary_count_(0) {}
-
-  Status AppendValue(PyObject* obj) override {
-    if (Strict) {
-      // raise if the object is not unicode or not an utf-8 encoded bytes
-      ARROW_ASSIGN_OR_RAISE(this->string_view_, ValueConverter<Type>::FromPython(obj));
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
     } else {
-      // keep track of whether values are unicode or bytes; if any bytes are
-      // observe, the result will be bytes
-      bool is_utf8;
-      ARROW_ASSIGN_OR_RAISE(this->string_view_,
-                            ValueConverter<Type>::FromPython(obj, &is_utf8));
-      if (!is_utf8) {
-        ++binary_count_;
-      }
+      ARROW_ASSIGN_OR_RAISE(auto converted,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(converted);
     }
-    return this->AppendString(this->string_view_);
   }
+};
 
-  Status GetResult(std::shared_ptr<ChunkedArray>* out) override {
-    RETURN_NOT_OK(SeqConverter::GetResult(out));
-
-    // If we saw any non-unicode, cast results to BinaryArray
-    if (binary_count_) {
-      // We should have bailed out earlier
-      DCHECK(!Strict);
-      auto binary_type = TypeTraits<typename Type::PhysicalType>::type_singleton();
-      return (*out)->View(binary_type).Value(out);
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_string_view<U>>
+    : public DictionaryConverter<U, PyConverter> {
+ public:
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(auto view,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(util::string_view(view.bytes, view.size));
     }
-    return Status::OK();
   }
-
- protected:
-  int64_t binary_count_;
 };
 
-// ----------------------------------------------------------------------
-// Convert lists (NumPy arrays containing lists or ndarrays as values)
-
 // If the value type does not match the expected NumPy dtype, then fall through
 // to a slower PySequence-based path
-#define LIST_FAST_CASE(TYPE, NUMPY_TYPE, ArrowType)            \
-  case Type::TYPE: {                                           \
-    if (PyArray_DESCR(arr)->type_num != NUMPY_TYPE) {          \
-      return value_converter_->Extend(obj, value_length);      \
-    }                                                          \
-    return AppendNdarrayTypedItem<NUMPY_TYPE, ArrowType>(arr); \
+#define LIST_FAST_CASE(TYPE_ID, TYPE, NUMPY_TYPE)               \
+  case Type::TYPE_ID: {                                         \
+    if (PyArray_DESCR(ndarray)->type_num != NUMPY_TYPE) {       \
+      return Extend(this->value_converter_.get(), value, size); \
+    }                                                           \
+    return AppendNdarrayTyped<TYPE, NUMPY_TYPE>(ndarray);       \
   }
 
 // Use internal::VisitSequence, fast for NPY_OBJECT but slower otherwise
-#define LIST_SLOW_CASE(TYPE)                            \
-  case Type::TYPE: {                                    \
-    return value_converter_->Extend(obj, value_length); \
+#define LIST_SLOW_CASE(TYPE_ID)                               \
+  case Type::TYPE_ID: {                                       \
+    return Extend(this->value_converter_.get(), value, size); \
   }
 
-// Base class for ListConverter and FixedSizeListConverter (to have both work with CRTP)
-template <typename TypeClass, NullCoding null_coding>
-class BaseListConverter : public TypedConverter<TypeClass, null_coding> {
+template <typename T>
+class PyListConverter : public ListConverter<T, PyConverter, PyConverterTrait> {
  public:
-  using BuilderType = typename TypeTraits<TypeClass>::BuilderType;
-
-  explicit BaseListConverter(bool from_pandas, bool strict_conversions,
-                             bool ignore_timezone)
-      : from_pandas_(from_pandas),
-        strict_conversions_(strict_conversions),
-        ignore_timezone_(ignore_timezone) {}
-
-  Status Init(ArrayBuilder* builder) override {
-    this->builder_ = builder;
-    this->typed_builder_ = checked_cast<BuilderType*>(builder);
-
-    this->value_type_ = checked_cast<const TypeClass&>(*builder->type()).value_type();
-    RETURN_NOT_OK(GetConverter(value_type_, from_pandas_, strict_conversions_,
-                               ignore_timezone_, &value_converter_));
-    return this->value_converter_->Init(this->typed_builder_->value_builder());
-  }
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->list_builder_->AppendNull();
+    }
 
-  template <int NUMPY_TYPE, typename Type>
-  Status AppendNdarrayTypedItem(PyArrayObject* arr) {
-    using traits = internal::npy_traits<NUMPY_TYPE>;
-    using T = typename traits::value_type;
-    using ValueBuilderType = typename TypeTraits<Type>::BuilderType;
+    RETURN_NOT_OK(this->list_builder_->Append());
+    if (PyArray_Check(value)) {
+      RETURN_NOT_OK(AppendNdarray(value));
+    } else if (PySequence_Check(value)) {
+      RETURN_NOT_OK(AppendSequence(value));
+    } else {
+      return internal::InvalidType(
+          value, "was not a sequence or recognized null for conversion to list type");
+    }
 
-    const bool null_sentinels_possible =
-        // Always treat Numpy's NaT as null
-        NUMPY_TYPE == NPY_DATETIME || NUMPY_TYPE == NPY_TIMEDELTA ||
-        // Observing pandas's null sentinels
-        (from_pandas_ && traits::supports_nulls);
+    return ValidateBuilder(this->list_type_);
+  }
 
-    auto child_builder = checked_cast<ValueBuilderType*>(value_converter_->builder());
+ protected:
+  Status ValidateOverflow(const MapType*, int64_t size) { return Status::OK(); }

Review comment:
       Since MapArray has the same offset structure as ListArray it should probably have non empty overflow validation

##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -329,985 +302,602 @@ struct ValueConverter<DurationType> {
         default:
           return Status::UnknownError("Invalid time unit");
       }
+    } else if (PyArray_CheckAnyScalarExact(obj)) {
+      // validate that the numpy scalar has np.datetime64 dtype
+      std::shared_ptr<DataType> numpy_type;
+      RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &numpy_type));
+      if (!numpy_type->Equals(*type)) {
+        return Status::NotImplemented("Expected np.timedelta64 but got: ",
+                                      numpy_type->ToString());
+      }
+      return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
     } else {
       RETURN_NOT_OK(internal::CIntFromPython(obj, &value));
     }
     return value;
   }
 
-  static inline Result<int64_t> FromNumpy(PyObject* obj, TimeUnit::type unit) {
-    // validate that the numpy scalar has np.timedelta64 dtype
-    std::shared_ptr<DataType> type;
-    RETURN_NOT_OK(NumPyDtypeToArrow(PyArray_DescrFromScalar(obj), &type));
-    if (type->id() != DurationType::type_id) {
-      // TODO(kszucs): the message should highlight the received numpy dtype
-      return Status::Invalid("Expected np.timedelta64 but got: ", type->ToString());
-    }
-    // validate that the time units are matching
-    if (unit != checked_cast<const DurationType&>(*type).unit()) {
-      return Status::NotImplemented(
-          "Cannot convert NumPy np.timedelta64 objects with differing unit");
-    }
-    // convert the numpy value
-    return reinterpret_cast<PyTimedeltaScalarObject*>(obj)->obval;
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_any_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    return std::move(view);
-  }
-};
+  // The binary-like intermediate representation is PyBytesView because it keeps temporary
+  // python objects alive (non-contiguous memoryview) and stores whether the original
+  // object was unicode encoded or not, which is used for unicode -> bytes coersion if
+  // there is a non-unicode object observed.
 
-template <typename Type>
-struct ValueConverter<Type, enable_if_string_like<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj) {
-    // strict conversion, force output to be unicode / utf8 and validate that
-    // any binary values are utf8
-    bool is_utf8 = false;
-    PyBytesView view;
-
-    RETURN_NOT_OK(view.FromString(obj, &is_utf8));
-    if (!is_utf8) {
-      return internal::InvalidValue(obj, "was not a utf8 string");
-    }
-    return std::move(view);
+  static Result<PyBytesView> Convert(const BaseBinaryType*, const O&, I obj) {
+    return PyBytesView::FromString(obj);
   }
 
-  static inline Result<PyBytesView> FromPython(PyObject* obj, bool* is_utf8) {
-    PyBytesView view;
-
-    // Non-strict conversion; keep track of whether values are unicode or bytes
-    if (PyUnicode_Check(obj)) {
-      *is_utf8 = true;
-      RETURN_NOT_OK(view.FromUnicode(obj));
+  static Result<PyBytesView> Convert(const FixedSizeBinaryType* type, const O&, I obj) {
+    ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj));
+    if (ARROW_PREDICT_TRUE(view.size == type->byte_width())) {
+      return std::move(view);
     } else {
-      // If not unicode or bytes, FromBinary will error
-      *is_utf8 = false;
-      RETURN_NOT_OK(view.FromBinary(obj));
-    }
-    return std::move(view);
-  }
-};
-
-template <typename Type>
-struct ValueConverter<Type, enable_if_fixed_size_binary<Type>> {
-  static inline Result<PyBytesView> FromPython(PyObject* obj, int32_t byte_width) {
-    PyBytesView view;
-    RETURN_NOT_OK(view.FromString(obj));
-    if (ARROW_PREDICT_FALSE(view.size != byte_width)) {
       std::stringstream ss;
-      ss << "expected to be length " << byte_width << " was " << view.size;
+      ss << "expected to be length " << type->byte_width() << " was " << view.size;
       return internal::InvalidValue(obj, ss.str());
-    } else {
-      return std::move(view);
     }
   }
-};
-
-// ----------------------------------------------------------------------
-// Sequence converter base and CRTP "middle" subclasses
-
-class SeqConverter;
-
-// Forward-declare converter factory
-Status GetConverter(const std::shared_ptr<DataType>& type, bool from_pandas,
-                    bool strict_conversions, bool ignore_timezone,
-                    std::unique_ptr<SeqConverter>* out);
-
-// Marshal Python sequence (list, tuple, etc.) to Arrow array
-class SeqConverter {
- public:
-  virtual ~SeqConverter() = default;
-
-  // Initialize the sequence converter with an ArrayBuilder created
-  // externally. The reason for this interface is that we have
-  // arrow::MakeBuilder which also creates child builders for nested types, so
-  // we have to pass in the child builders to child SeqConverter in the case of
-  // converting Python objects to Arrow nested types
-  virtual Status Init(ArrayBuilder* builder) = 0;
-
-  // Append a single null value to the builder
-  virtual Status AppendNull() = 0;
-
-  // Append a valid value
-  virtual Status AppendValue(PyObject* seq) = 0;
-
-  // Append a single python object handling Null values
-  virtual Status Append(PyObject* seq) = 0;
 
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status Extend(PyObject* seq, int64_t size) = 0;
-
-  // Append the contents of a Python sequence to the underlying builder,
-  // virtual version
-  virtual Status ExtendMasked(PyObject* seq, PyObject* mask, int64_t size) = 0;
-
-  virtual Status Close() {
-    if (chunks_.size() == 0 || builder_->length() > 0) {
-      std::shared_ptr<Array> last_chunk;
-      RETURN_NOT_OK(builder_->Finish(&last_chunk));
-      chunks_.emplace_back(std::move(last_chunk));
+  template <typename T>
+  static enable_if_string<T, Result<PyBytesView>> Convert(const T*, const O& options,
+                                                          I obj) {
+    if (options.strict) {
+      // Strict conversion, force output to be unicode / utf8 and validate that
+      // any binary values are utf8
+      ARROW_ASSIGN_OR_RAISE(auto view, PyBytesView::FromString(obj, true));
+      if (!view.is_utf8) {
+        return internal::InvalidValue(obj, "was not a utf8 string");
+      }
+      return std::move(view);
+    } else {
+      // Non-strict conversion; keep track of whether values are unicode or bytes
+      return PyBytesView::FromString(obj);
     }
-    return Status::OK();
   }
 
-  virtual Status GetResult(std::shared_ptr<ChunkedArray>* out) {
-    // Still some accumulated data in the builder. If there are no chunks, we
-    // always call Finish to deal with the edge case where a size-0 sequence
-    // was converted with a specific output type, like array([], type=t)
-    RETURN_NOT_OK(Close());
-    *out = std::make_shared<ChunkedArray>(this->chunks_, builder_->type());
-    return Status::OK();
+  static Result<bool> Convert(const DataType* type, const O&, I obj) {
+    return Status::NotImplemented("PyValue::Convert is not implemented for type ", type);
   }
-
-  ArrayBuilder* builder() const { return builder_; }
-
-  int num_chunks() const { return static_cast<int>(chunks_.size()); }
-
- protected:
-  ArrayBuilder* builder_;
-  bool unfinished_builder_;
-  std::vector<std::shared_ptr<Array>> chunks_;
 };
 
-template <typename Type, NullCoding null_coding = NullCoding::NONE_ONLY>
-class TypedConverter : public SeqConverter {
- public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  Status Init(ArrayBuilder* builder) override {
-    builder_ = builder;
-    DCHECK_NE(builder_, nullptr);
-    typed_builder_ = checked_cast<BuilderType*>(builder);
-    return Status::OK();
-  }
-
-  // Append a missing item (default implementation)
-  Status AppendNull() override { return this->typed_builder_->AppendNull(); }
-
-  // Append null if the obj is None or pandas null otherwise the valid value
-  Status Append(PyObject* obj) override {
-    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
-  }
-
-  Status Extend(PyObject* obj, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequence(
-        obj, [this](PyObject* item, bool* /* unused */) { return this->Append(item); });
-  }
-
-  Status ExtendMasked(PyObject* obj, PyObject* mask, int64_t size) override {
-    /// Ensure we've allocated enough space
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    // Iterate over the items adding each one
-    return internal::VisitSequenceMasked(
-        obj, mask, [this](PyObject* item, bool is_masked, bool* /* unused */) {
-          if (is_masked) {
-            return this->AppendNull();
-          } else {
-            // This will also apply the null-checking convention in the event
-            // that the value is not masked
-            return this->Append(item);  // perhaps use AppendValue instead?
-          }
-        });
-  }
+template <typename T>
+Status Extend(T* converter, PyObject* values, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequence(values, [converter](PyObject* item, bool* /* unused */) {
+    return converter->Append(item);
+  });
+}
 
- protected:
-  BuilderType* typed_builder_;
-};
+// Convert and append a sequence of values masked with a numpy array
+template <typename T>
+Status ExtendMasked(T* converter, PyObject* values, PyObject* mask, int64_t size) {
+  /// Ensure we've allocated enough space
+  RETURN_NOT_OK(converter->Reserve(size));
+  // Iterate over the items adding each one
+  return internal::VisitSequenceMasked(
+      values, mask, [converter](PyObject* item, bool is_masked, bool* /* unused */) {
+        if (is_masked) {
+          return converter->AppendNull();
+        } else {
+          // This will also apply the null-checking convention in the event
+          // that the value is not masked
+          return converter->Append(item);  // perhaps use AppendValue instead?
+        }
+      });
+}
 
-// ----------------------------------------------------------------------
-// Sequence converter for null type
+// The base Converter class is a mixin with predefined behavior and constructors.
+using PyConverter = Converter<PyObject*, PyConversionOptions>;
 
-template <NullCoding null_coding>
-class NullConverter : public TypedConverter<NullType, null_coding> {
- public:
-  Status AppendValue(PyObject* obj) override {
-    return internal::InvalidValue(obj, "converting to null type");
-  }
-};
+template <typename T, typename Enable = void>
+class PyPrimitiveConverter;
 
-// ----------------------------------------------------------------------
-// Sequence converter template for primitive (integer and floating point bool) types
+template <typename T>
+class PyListConverter;
 
-template <typename Type, NullCoding null_coding>
-class PrimitiveConverter : public TypedConverter<Type, null_coding> {
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value, ValueConverter<Type>::FromPython(obj));
-    return this->typed_builder_->Append(value);
-  }
-};
+template <typename U, typename Enable = void>
+class PyDictionaryConverter;
 
-// ----------------------------------------------------------------------
-// Sequence converters for temporal types
+class PyStructConverter;
 
-template <typename Type, NullCoding null_coding>
-class TimeConverter : public TypedConverter<Type, null_coding> {
- public:
-  explicit TimeConverter(TimeUnit::type unit, bool ignore_timezone)
-      : unit_(unit), ignore_timezone_(ignore_timezone) {}
-
-  // TODO(kszucs): support numpy values for date and time converters
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto value,
-                          ValueConverter<Type>::FromPython(obj, unit_, ignore_timezone_));
-    return this->typed_builder_->Append(value);
-  }
+template <typename T, typename Enable = void>
+struct PyConverterTrait;
 
- protected:
-  TimeUnit::type unit_;
-  bool ignore_timezone_;
+template <typename T>
+struct PyConverterTrait<
+    T, enable_if_t<!is_nested_type<T>::value && !is_interval_type<T>::value &&
+                   !is_extension_type<T>::value>> {
+  using type = PyPrimitiveConverter<T>;
 };
 
-// TODO(kszucs): move it to the type_traits
 template <typename T>
-struct NumpyType {};
+struct PyConverterTrait<T, enable_if_list_like<T>> {
+  using type = PyListConverter<T>;
+};
 
 template <>
-struct NumpyType<TimestampType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_DATETIME>::isnull(v);
-  }
+struct PyConverterTrait<StructType> {
+  using type = PyStructConverter;
 };
 
 template <>
-struct NumpyType<DurationType> {
-  static inline bool isnull(int64_t v) {
-    return internal::npy_traits<NPY_TIMEDELTA>::isnull(v);
-  }
+struct PyConverterTrait<DictionaryType> {
+  template <typename T>
+  using type = PyDictionaryConverter<T>;
 };
 
-template <typename Type, NullCoding null_coding>
-class TemporalConverter : public TimeConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_null_type<T>::value || is_boolean_type<T>::value ||
+                   is_number_type<T>::value || is_decimal_type<T>::value ||
+                   is_date_type<T>::value || is_time_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using TimeConverter<Type, null_coding>::TimeConverter;
-
-  Status AppendValue(PyObject* obj) override {
-    int64_t value;
-    if (PyArray_CheckAnyScalarExact(obj)) {
-      // convert np.datetime64 / np.timedelta64 depending on Type
-      ARROW_ASSIGN_OR_RAISE(value, ValueConverter<Type>::FromNumpy(obj, this->unit_));
-      if (NumpyType<Type>::isnull(value)) {
-        // checks numpy NaT sentinel after conversion
-        return this->typed_builder_->AppendNull();
-      }
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
     } else {
       ARROW_ASSIGN_OR_RAISE(
-          value,
-          ValueConverter<Type>::FromPython(
-              obj, this->unit_, TimeConverter<Type, null_coding>::ignore_timezone_));
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      return this->primitive_builder_->Append(converted);
     }
-    return this->typed_builder_->Append(value);
   }
 };
 
-// ----------------------------------------------------------------------
-// Sequence converters for Binary, FixedSizeBinary, String
-
-template <typename Type, NullCoding null_coding>
-class BinaryLikeConverter : public TypedConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<
+    T, enable_if_t<is_timestamp_type<T>::value || is_duration_type<T>::value>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  using BuilderType = typename TypeTraits<Type>::BuilderType;
-
-  inline Status AutoChunk(Py_ssize_t size) {
-    // did we reach the builder size limit?
-    if (ARROW_PREDICT_FALSE(this->typed_builder_->value_data_length() + size >
-                            BuilderType::memory_limit())) {
-      // builder would be full, so need to add a new chunk
-      std::shared_ptr<Array> chunk;
-      RETURN_NOT_OK(this->typed_builder_->Finish(&chunk));
-      this->chunks_.emplace_back(std::move(chunk));
-    }
-    return Status::OK();
-  }
-
-  Status AppendString(const PyBytesView& view) {
-    // check that the value fits in the datatype
-    if (view.size > BuilderType::memory_limit()) {
-      return Status::Invalid("string too large for datatype");
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto converted, PyValue::Convert(this->primitive_type_, this->options_, value));
+      // Numpy NaT sentinels can be checked after the conversion
+      if (PyArray_CheckAnyScalarExact(value) &&
+          PyValue::IsNaT(this->primitive_type_, converted)) {
+        return this->primitive_builder_->AppendNull();
+      } else {
+        return this->primitive_builder_->Append(converted);
+      }
     }
-    DCHECK_GE(view.size, 0);
-
-    // create a new chunk if the value would overflow the builder
-    RETURN_NOT_OK(AutoChunk(view.size));
-
-    // now we can safely append the value to the builder
-    RETURN_NOT_OK(
-        this->typed_builder_->Append(::arrow::util::string_view(view.bytes, view.size)));
-
-    return Status::OK();
   }
-
- protected:
-  // Create a single instance of PyBytesView here to prevent unnecessary object
-  // creation/destruction
-  PyBytesView string_view_;
 };
 
-template <typename Type, NullCoding null_coding>
-class BinaryConverter : public BinaryLikeConverter<Type, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_binary<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(auto view, ValueConverter<Type>::FromPython(obj));
-    return this->AppendString(view);
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
+    }
   }
 };
 
-template <NullCoding null_coding>
-class FixedSizeBinaryConverter
-    : public BinaryLikeConverter<FixedSizeBinaryType, null_coding> {
+template <typename T>
+class PyPrimitiveConverter<T, enable_if_string_like<T>>
+    : public PrimitiveConverter<T, PyConverter> {
  public:
-  explicit FixedSizeBinaryConverter(int32_t byte_width) : byte_width_(byte_width) {}
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->primitive_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(
+          auto view, PyValue::Convert(this->primitive_type_, this->options_, value));
+      if (!view.is_utf8) {
+        // observed binary value
+        observed_binary_ = true;
+      }
+      ARROW_RETURN_NOT_OK(this->primitive_builder_->ValidateOverflow(view.size));
+      return this->primitive_builder_->Append(util::string_view(view.bytes, view.size));
+    }
+  }
 
-  Status AppendValue(PyObject* obj) override {
-    ARROW_ASSIGN_OR_RAISE(
-        this->string_view_,
-        ValueConverter<FixedSizeBinaryType>::FromPython(obj, byte_width_));
-    return this->AppendString(this->string_view_);
+  Result<std::shared_ptr<Array>> ToArray() override {
+    ARROW_ASSIGN_OR_RAISE(auto array, (PrimitiveConverter<T, PyConverter>::ToArray()));
+    if (observed_binary_) {
+      // if we saw any non-unicode, cast results to BinaryArray
+      auto binary_type = TypeTraits<typename T::PhysicalType>::type_singleton();
+      return array->View(binary_type);
+    } else {
+      return array;
+    }
   }
 
  protected:
-  int32_t byte_width_;
+  bool observed_binary_ = false;
 };
 
-// For String/UTF8, if strict_conversions enabled, we reject any non-UTF8,
-// otherwise we allow but return results as BinaryArray
-template <typename Type, bool Strict, NullCoding null_coding>
-class StringConverter : public BinaryLikeConverter<Type, null_coding> {
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_c_type<U>>
+    : public DictionaryConverter<U, PyConverter> {
  public:
-  StringConverter() : binary_count_(0) {}
-
-  Status AppendValue(PyObject* obj) override {
-    if (Strict) {
-      // raise if the object is not unicode or not an utf-8 encoded bytes
-      ARROW_ASSIGN_OR_RAISE(this->string_view_, ValueConverter<Type>::FromPython(obj));
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
     } else {
-      // keep track of whether values are unicode or bytes; if any bytes are
-      // observe, the result will be bytes
-      bool is_utf8;
-      ARROW_ASSIGN_OR_RAISE(this->string_view_,
-                            ValueConverter<Type>::FromPython(obj, &is_utf8));
-      if (!is_utf8) {
-        ++binary_count_;
-      }
+      ARROW_ASSIGN_OR_RAISE(auto converted,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(converted);
     }
-    return this->AppendString(this->string_view_);
   }
+};
 
-  Status GetResult(std::shared_ptr<ChunkedArray>* out) override {
-    RETURN_NOT_OK(SeqConverter::GetResult(out));
-
-    // If we saw any non-unicode, cast results to BinaryArray
-    if (binary_count_) {
-      // We should have bailed out earlier
-      DCHECK(!Strict);
-      auto binary_type = TypeTraits<typename Type::PhysicalType>::type_singleton();
-      return (*out)->View(binary_type).Value(out);
+template <typename U>
+class PyDictionaryConverter<U, enable_if_has_string_view<U>>
+    : public DictionaryConverter<U, PyConverter> {
+ public:
+  Status Append(PyObject* value) override {
+    if (PyValue::IsNull(this->options_, value)) {
+      return this->value_builder_->AppendNull();
+    } else {
+      ARROW_ASSIGN_OR_RAISE(auto view,
+                            PyValue::Convert(this->value_type_, this->options_, value));
+      return this->value_builder_->Append(util::string_view(view.bytes, view.size));
     }
-    return Status::OK();
   }
-
- protected:
-  int64_t binary_count_;
 };
 
-// ----------------------------------------------------------------------
-// Convert lists (NumPy arrays containing lists or ndarrays as values)
-
 // If the value type does not match the expected NumPy dtype, then fall through
 // to a slower PySequence-based path
-#define LIST_FAST_CASE(TYPE, NUMPY_TYPE, ArrowType)            \
-  case Type::TYPE: {                                           \
-    if (PyArray_DESCR(arr)->type_num != NUMPY_TYPE) {          \
-      return value_converter_->Extend(obj, value_length);      \
-    }                                                          \
-    return AppendNdarrayTypedItem<NUMPY_TYPE, ArrowType>(arr); \
+#define LIST_FAST_CASE(TYPE_ID, TYPE, NUMPY_TYPE)               \
+  case Type::TYPE_ID: {                                         \
+    if (PyArray_DESCR(ndarray)->type_num != NUMPY_TYPE) {       \
+      return Extend(this->value_converter_.get(), value, size); \
+    }                                                           \
+    return AppendNdarrayTyped<TYPE, NUMPY_TYPE>(ndarray);       \
   }
 
 // Use internal::VisitSequence, fast for NPY_OBJECT but slower otherwise
-#define LIST_SLOW_CASE(TYPE)                            \
-  case Type::TYPE: {                                    \
-    return value_converter_->Extend(obj, value_length); \
+#define LIST_SLOW_CASE(TYPE_ID)                               \
+  case Type::TYPE_ID: {                                       \
+    return Extend(this->value_converter_.get(), value, size); \
   }

Review comment:
       Since these macros are only used in PyListConverter::AppendNdarray, please #define and #undef them inside that scope




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API [WIP]

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



##########
File path: cpp/src/arrow/ipc/metadata_internal.cc
##########
@@ -1159,7 +1159,7 @@ Status GetKeyValueMetadata(const KVVector* fb_metadata,
   auto metadata = std::make_shared<KeyValueMetadata>();
 
   metadata->reserve(fb_metadata->size());
-  for (const auto& pair : *fb_metadata) {
+  for (const auto pair : *fb_metadata) {

Review comment:
       My local toolchain always complains 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.

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,353 @@
+// 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 <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+namespace internal {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename T, typename BaseConverter>
+class PrimitiveConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    primitive_type_ = checked_cast<const T*>(this->type_.get());
+    primitive_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const T* primitive_type_;
+  BuilderType* primitive_builder_;
+};
+
+template <typename T, typename BaseConverter>
+class ListConverter : public BaseConverter {
+ public:
+  using BuilderType = typename TypeTraits<T>::BuilderType;
+
+  Status Init() override {
+    list_type_ = checked_cast<const T*>(this->type_.get());
+    list_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    value_converter_ = this->children_[0];
+    return Status::OK();
+  }
+
+ protected:
+  const T* list_type_;
+  BuilderType* list_builder_;
+  std::shared_ptr<BaseConverter> value_converter_;
+};
+
+template <typename BaseConverter>
+class StructConverter : public BaseConverter {
+ public:
+  Status Init() override {
+    struct_type_ = checked_cast<const StructType*>(this->type_.get());
+    struct_builder_ = checked_cast<StructBuilder*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const StructType* struct_type_;
+  StructBuilder* struct_builder_;
+};
+
+template <typename U, typename BaseConverter>
+class DictionaryConverter : public BaseConverter {
+ public:
+  using BuilderType = DictionaryBuilder<U>;
+
+  Status Init() override {
+    dict_type_ = checked_cast<const DictionaryType*>(this->type_.get());
+    value_type_ = checked_cast<const U*>(dict_type_->value_type().get());
+    value_builder_ = checked_cast<BuilderType*>(this->builder_.get());
+    return Status::OK();
+  }
+
+ protected:
+  const DictionaryType* dict_type_;
+  const U* value_type_;
+  BuilderType* value_builder_;
+};
+
+template <typename Converter>
+struct MakeConverterImpl;
+
+template <typename Input, typename Options, typename Self>
+class Converter {
+ public:
+  using InputType = Input;
+  using OptionsType = Options;
+
+  template <typename T>
+  using Primitive = PrimitiveConverter<T, Self>;
+  template <typename T>
+  using List = ListConverter<T, Self>;
+  template <typename T>
+  using Dictionary = DictionaryConverter<T, Self>;
+  using Struct = StructConverter<Self>;
+
+  static Result<std::shared_ptr<Self>> Make(std::shared_ptr<DataType> type,
+                                            MemoryPool* pool, OptionsType options) {
+    std::shared_ptr<Self> out;
+    MakeConverterImpl<Self> visitor = {type, pool, options, &out};
+    ARROW_RETURN_NOT_OK(VisitTypeInline(*type, &visitor));
+    ARROW_RETURN_NOT_OK(out->Init());
+    return out;
+  }
+
+  virtual ~Converter() = default;
+
+  virtual Status Init() { return Status::OK(); }
+
+  virtual Status Append(InputType value) {
+    return Status::NotImplemented("Converter not implemented for type ",
+                                  type()->ToString());
+  }

Review comment:
       Ok, updating. 




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,248 @@
+// 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 <datetime.h>
+
+#include <algorithm>
+#include <iostream>
+#include <limits>
+#include <map>
+#include <sstream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/decimal.h"
+#include "arrow/util/int_util_internal.h"
+#include "arrow/util/logging.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename I, typename O>

Review comment:
       Updated.




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

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



[GitHub] [arrow] pitrou commented on a change in pull request #8088: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API [WIP]

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



##########
File path: cpp/src/arrow/ipc/metadata_internal.cc
##########
@@ -1159,7 +1159,7 @@ Status GetKeyValueMetadata(const KVVector* fb_metadata,
   auto metadata = std::make_shared<KeyValueMetadata>();
 
   metadata->reserve(fb_metadata->size());
-  for (const auto& pair : *fb_metadata) {
+  for (const auto pair : *fb_metadata) {

Review comment:
       Was there a problem here?

##########
File path: cpp/src/arrow/array/builder_base.h
##########
@@ -56,6 +56,8 @@ class ARROW_EXPORT ArrayBuilder {
   /// skip shared pointers and just return a raw pointer
   ArrayBuilder* child(int i) { return children_[i].get(); }
 
+  std::shared_ptr<ArrayBuilder> child_builder(int i) const { return children_[i]; }

Review comment:
       `const shared_ptr&` perhaps?

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,248 @@
+// 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 <datetime.h>
+
+#include <algorithm>
+#include <iostream>
+#include <limits>
+#include <map>
+#include <sstream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/decimal.h"
+#include "arrow/util/int_util_internal.h"
+#include "arrow/util/logging.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename I, typename O>

Review comment:
       Please avoid cryptic one-letter names. Instead `InputType`, `OptionsType`.

##########
File path: cpp/src/arrow/util/converter.h
##########
@@ -0,0 +1,248 @@
+// 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 <datetime.h>
+
+#include <algorithm>
+#include <iostream>
+#include <limits>
+#include <map>
+#include <sstream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/status.h"
+#include "arrow/type.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/checked_cast.h"
+#include "arrow/util/decimal.h"
+#include "arrow/util/int_util_internal.h"
+#include "arrow/util/logging.h"
+
+#include "arrow/visitor_inline.h"
+
+namespace arrow {
+
+using internal::checked_cast;
+using internal::checked_pointer_cast;
+
+template <typename I, typename O>
+class ARROW_EXPORT ArrayConverter {
+ public:
+  using InputType = I;
+  using OptionsType = O;
+
+  ArrayConverter(const std::shared_ptr<DataType>& type,
+                 std::shared_ptr<ArrayBuilder> builder, O options)
+      : sp_type_(type), sp_builder_(builder), options_(options) {}
+
+  virtual ~ArrayConverter() = default;
+  std::shared_ptr<ArrayBuilder> builder() { return sp_builder_; }
+  std::shared_ptr<ArrayBuilder> type() { return sp_type_; }
+  O options() { return options_; }
+
+  virtual Status Init() { return Status::OK(); };
+  virtual Status Reserve(int64_t additional_capacity) = 0;
+
+  virtual Status Append(I value) = 0;
+  virtual Status AppendNull() = 0;
+
+  virtual Status Extend(I seq, int64_t size) = 0;
+
+  virtual Result<std::shared_ptr<Array>> Finish() = 0;
+
+  // virtual Result<std::shared_ptr<Array>> ToArray(I value);
+  // virtual Result<std::shared_ptr<ChunkedArray>> ToChunkedArray(I value);
+
+ protected:
+  const std::shared_ptr<DataType> sp_type_;
+  std::shared_ptr<ArrayBuilder> sp_builder_;
+  O options_;
+};
+
+template <typename T, typename AC>
+class ARROW_EXPORT TypedArrayConverter : public AC {

Review comment:
       I don't expect to have to pass the base class myself as a template argument. It should be inferred?




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

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



[GitHub] [arrow] bkietz commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/array/array_list_test.cc
##########
@@ -508,6 +534,8 @@ TYPED_TEST(TestListArray, ValidateOffsets) { this->TestValidateOffsets(); }
 
 TYPED_TEST(TestListArray, CornerCases) { this->TestCornerCases(); }
 
+TYPED_TEST(TestListArray, TestOverflowCheck) { this->TestOverflowCheck(); }

Review comment:
       ```suggestion
   TYPED_TEST(TestListArray, LARGE_MEMORY_TEST(TestOverflowCheck)) { this->TestOverflowCheck(); }
   ```




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

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



[GitHub] [arrow] kszucs commented on pull request #8088: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API [WIP]

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


   Not directly, but this will help to solve multiple ones. I'm going to create one.


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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: cpp/src/arrow/python/python_test.cc
##########
@@ -422,17 +421,15 @@ TEST_F(DecimalTest, TestNoneAndNaN) {
   ASSERT_EQ(0, PyList_SetItem(list, 2, missing_value2));
   ASSERT_EQ(0, PyList_SetItem(list, 3, missing_value3));
 
-  std::shared_ptr<ChunkedArray> arr, arr_from_pandas;
   PyConversionOptions options;
-  ASSERT_RAISES(TypeError, ConvertPySequence(list, options, &arr));
+  ASSERT_RAISES(TypeError, ConvertPySequence(list, nullptr, options));
 
   options.from_pandas = true;
-  ASSERT_OK(ConvertPySequence(list, options, &arr_from_pandas));
-  auto c0 = arr_from_pandas->chunk(0);
-  ASSERT_TRUE(c0->IsValid(0));
-  ASSERT_TRUE(c0->IsNull(1));
-  ASSERT_TRUE(c0->IsNull(2));
-  ASSERT_TRUE(c0->IsNull(3));
+  auto arr = ConvertPySequence(list, nullptr, options).ValueOrDie();

Review comment:
       Updating.




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8088: ARROW-9992: [C++][Python] Refactor python to arrow conversions based on a reusable conversion API

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



##########
File path: python/pyarrow/tests/test_convert_builtin.py
##########
@@ -132,6 +133,10 @@ def _as_tuple(xs):
     return tuple(xs)
 
 
+def _as_pairs(xs):

Review comment:
       Not used anymore.




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

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