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/09/19 02:36:57 UTC

[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

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