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

[GitHub] [arrow] kszucs opened a new pull request #8008: ARROW-9369: [Python] Support conversion from python sequence to dictionary type

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


   


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

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



[GitHub] [arrow] pitrou commented on a change in pull request #8008: ARROW-9369: [Python] Support conversion from python sequence to dictionary type

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



##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -903,6 +897,75 @@ class FixedSizeListConverter : public BaseListConverter<FixedSizeListType, null_
   int64_t list_size_;
 };
 
+// ----------------------------------------------------------------------
+// Convert dictionary
+
+template <typename ValueType, NullCoding null_coding>
+class DictionaryConverter : public SeqConverter {
+ public:
+  using BuilderType = DictionaryBuilder<ValueType>;
+
+  Status Init(ArrayBuilder* builder) override {
+    RETURN_NOT_OK(SeqConverter::Init(builder));
+    typed_builder_ = checked_cast<BuilderType*>(builder);
+    return Status::OK();
+  }
+
+  Status Append(PyObject* obj) override {
+    // Append null if the obj is None or pandas null otherwise the valid value
+    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
+  }
+
+ protected:
+  BuilderType* typed_builder_;
+};
+
+template <typename ValueType, NullCoding null_coding>
+class PrimitiveDictionaryConverter : public DictionaryConverter<ValueType, null_coding> {
+ public:
+  Status AppendValue(PyObject* obj) override {
+    ARROW_ASSIGN_OR_RAISE(auto value, ValueConverter<ValueType>::FromPython(obj));
+    return this->typed_builder_->Append(value);
+  }
+};
+
+template <typename ValueType, NullCoding null_coding>
+class BinaryLikeDictionaryConverter : public DictionaryConverter<ValueType, null_coding> {
+ public:
+  Status AppendValue(PyObject* obj) override {
+    ARROW_ASSIGN_OR_RAISE(string_view_, ValueConverter<ValueType>::FromPython(obj));
+    // DCHECK_GE(string_view_.size, 0);
+    RETURN_NOT_OK(this->typed_builder_->Append(string_view_.bytes,

Review comment:
       Isn't there a `TypedBuilder::Append(string_view)`? If not, can you please add it?

##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -903,6 +897,75 @@ class FixedSizeListConverter : public BaseListConverter<FixedSizeListType, null_
   int64_t list_size_;
 };
 
+// ----------------------------------------------------------------------
+// Convert dictionary
+
+template <typename ValueType, NullCoding null_coding>
+class DictionaryConverter : public SeqConverter {
+ public:
+  using BuilderType = DictionaryBuilder<ValueType>;
+
+  Status Init(ArrayBuilder* builder) override {
+    RETURN_NOT_OK(SeqConverter::Init(builder));
+    typed_builder_ = checked_cast<BuilderType*>(builder);
+    return Status::OK();
+  }
+
+  Status Append(PyObject* obj) override {
+    // Append null if the obj is None or pandas null otherwise the valid value
+    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
+  }
+
+ protected:
+  BuilderType* typed_builder_;
+};
+
+template <typename ValueType, NullCoding null_coding>
+class PrimitiveDictionaryConverter : public DictionaryConverter<ValueType, null_coding> {
+ public:
+  Status AppendValue(PyObject* obj) override {
+    ARROW_ASSIGN_OR_RAISE(auto value, ValueConverter<ValueType>::FromPython(obj));
+    return this->typed_builder_->Append(value);
+  }
+};
+
+template <typename ValueType, NullCoding null_coding>
+class BinaryLikeDictionaryConverter : public DictionaryConverter<ValueType, null_coding> {
+ public:
+  Status AppendValue(PyObject* obj) override {
+    ARROW_ASSIGN_OR_RAISE(string_view_, ValueConverter<ValueType>::FromPython(obj));
+    // DCHECK_GE(string_view_.size, 0);

Review comment:
       Why?

##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -1123,6 +1186,53 @@ class DecimalConverter : public TypedConverter<arrow::Decimal128Type, null_codin
   std::shared_ptr<DecimalType> decimal_type_;
 };
 
+#define DICTIONARY_PRIMITIVE(TYPE_ENUM, TYPE_CLASS)                 \

Review comment:
       Please `#undef` at the end.

##########
File path: python/pyarrow/tests/test_scalars.py
##########
@@ -550,8 +550,8 @@ def test_dictionary():
         with pytest.warns(FutureWarning):
             assert s.dictionary_value.as_py() == v
 
-    with pytest.raises(pa.ArrowNotImplementedError):
-        pickle.loads(pickle.dumps(s))
+    restored = pickle.loads(pickle.dumps(s))
+    assert restored.equals(s)

Review comment:
       This should be inside the loop.
   (also, null values are not tested?)

##########
File path: python/pyarrow/tests/test_convert_builtin.py
##########
@@ -1640,3 +1640,85 @@ def test_map_from_tuples():
     for entry in [[(5,)], [()], [('5', 'foo', True)]]:
         with pytest.raises(ValueError, match="(?i)tuple size"):
             pa.array([entry], type=pa.map_('i4', 'i4'))
+
+
+def test_dictionary_from_boolean():
+    typ = pa.dictionary(pa.int8(), value_type=pa.bool_())
+    a = pa.array([False, False, True, False, True], type=typ)
+    assert isinstance(a.type, pa.DictionaryType)
+    assert a.type.equals(typ)
+
+    expected_indices = pa.array([0, 0, 1, 0, 1], type=pa.int8())
+    expected_dictionary = pa.array([False, True], type=pa.bool_())
+    assert a.indices.equals(expected_indices)
+    assert a.dictionary.equals(expected_dictionary)
+
+
+@pytest.mark.parametrize('value_type', [
+    pa.int8(),
+    pa.int16(),
+    pa.int32(),
+    pa.int64(),
+    pa.uint8(),
+    pa.uint16(),
+    pa.uint32(),
+    pa.uint64(),
+    pa.float32(),
+    pa.float64(),
+    pa.date32(),
+    pa.date64(),
+])
+def test_dictionary_from_integers(value_type):
+    typ = pa.dictionary(pa.int8(), value_type=value_type)
+    a = pa.array([1, 2, 1, 1, 2, 3], type=typ)
+    assert isinstance(a.type, pa.DictionaryType)
+    assert a.type.equals(typ)
+
+    expected_indices = pa.array([0, 1, 0, 0, 1, 2], type=pa.int8())
+    expected_dictionary = pa.array([1, 2, 3], type=value_type)
+    assert a.indices.equals(expected_indices)
+    assert a.dictionary.equals(expected_dictionary)
+
+
+@pytest.mark.parametrize('input_index_type', [
+    pa.int8(),
+    pa.int16(),
+    pa.int32(),
+    pa.int64()
+])
+def test_dictionary_is_always_adaptive(input_index_type):
+    # dictionary array is constructed using adaptive index type builder,
+    # meaning that the input index type is ignored since the output index
+    # type depends on the input data
+    typ = pa.dictionary(input_index_type, value_type=pa.int64())
+
+    a = pa.array(range(2**7), type=typ)
+    expected = pa.dictionary(pa.int8(), pa.int64())
+    assert a.type.equals(expected)
+
+    a = pa.array(range(2**7 + 1), type=typ)
+    expected = pa.dictionary(pa.int16(), pa.int64())
+    assert a.type.equals(expected)
+
+
+def test_dictionary_from_strings():
+    for value_type in [pa.binary(), pa.string()]:
+        typ = pa.dictionary(pa.int8(), value_type)
+        a = pa.array(["", "a", "bb", "a", "bb", "ccc"], type=typ)

Review comment:
       Also test with nulls?

##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -903,6 +897,75 @@ class FixedSizeListConverter : public BaseListConverter<FixedSizeListType, null_
   int64_t list_size_;
 };
 
+// ----------------------------------------------------------------------
+// Convert dictionary
+
+template <typename ValueType, NullCoding null_coding>
+class DictionaryConverter : public SeqConverter {
+ public:
+  using BuilderType = DictionaryBuilder<ValueType>;
+
+  Status Init(ArrayBuilder* builder) override {
+    RETURN_NOT_OK(SeqConverter::Init(builder));
+    typed_builder_ = checked_cast<BuilderType*>(builder);
+    return Status::OK();
+  }
+
+  Status Append(PyObject* obj) override {
+    // Append null if the obj is None or pandas null otherwise the valid value
+    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
+  }
+
+ protected:
+  BuilderType* typed_builder_;
+};
+
+template <typename ValueType, NullCoding null_coding>
+class PrimitiveDictionaryConverter : public DictionaryConverter<ValueType, null_coding> {
+ public:
+  Status AppendValue(PyObject* obj) override {
+    ARROW_ASSIGN_OR_RAISE(auto value, ValueConverter<ValueType>::FromPython(obj));
+    return this->typed_builder_->Append(value);
+  }
+};
+
+template <typename ValueType, NullCoding null_coding>
+class BinaryLikeDictionaryConverter : public DictionaryConverter<ValueType, null_coding> {
+ public:
+  Status AppendValue(PyObject* obj) override {
+    ARROW_ASSIGN_OR_RAISE(string_view_, ValueConverter<ValueType>::FromPython(obj));
+    // DCHECK_GE(string_view_.size, 0);
+    RETURN_NOT_OK(this->typed_builder_->Append(string_view_.bytes,
+                                               static_cast<int32_t>(string_view_.size)));
+    return Status::OK();
+  }
+
+ protected:
+  // Create a single instance of PyBytesView here to prevent unnecessary object
+  // creation/destruction
+  PyBytesView string_view_;
+};
+
+template <NullCoding null_coding>
+class FixedSizeBinaryDictionaryConverter
+    : public DictionaryConverter<FixedSizeBinaryType, null_coding> {
+ public:
+  explicit FixedSizeBinaryDictionaryConverter(int32_t byte_width)
+      : byte_width_(byte_width) {}
+
+  Status AppendValue(PyObject* obj) override {
+    ARROW_ASSIGN_OR_RAISE(
+        string_view_, ValueConverter<FixedSizeBinaryType>::FromPython(obj, byte_width_));
+    RETURN_NOT_OK(this->typed_builder_->Append(string_view_.bytes,
+                                               static_cast<int32_t>(string_view_.size)));
+    return Status::OK();
+  }
+
+ protected:
+  int32_t byte_width_;

Review comment:
       Add `const`

##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -903,6 +897,75 @@ class FixedSizeListConverter : public BaseListConverter<FixedSizeListType, null_
   int64_t list_size_;
 };
 
+// ----------------------------------------------------------------------
+// Convert dictionary
+
+template <typename ValueType, NullCoding null_coding>
+class DictionaryConverter : public SeqConverter {
+ public:
+  using BuilderType = DictionaryBuilder<ValueType>;
+
+  Status Init(ArrayBuilder* builder) override {
+    RETURN_NOT_OK(SeqConverter::Init(builder));
+    typed_builder_ = checked_cast<BuilderType*>(builder);
+    return Status::OK();
+  }
+
+  Status Append(PyObject* obj) override {
+    // Append null if the obj is None or pandas null otherwise the valid value
+    return NullChecker<null_coding>::Check(obj) ? AppendNull() : AppendValue(obj);
+  }
+
+ protected:
+  BuilderType* typed_builder_;
+};
+
+template <typename ValueType, NullCoding null_coding>
+class PrimitiveDictionaryConverter : public DictionaryConverter<ValueType, null_coding> {
+ public:
+  Status AppendValue(PyObject* obj) override {
+    ARROW_ASSIGN_OR_RAISE(auto value, ValueConverter<ValueType>::FromPython(obj));
+    return this->typed_builder_->Append(value);
+  }
+};
+
+template <typename ValueType, NullCoding null_coding>
+class BinaryLikeDictionaryConverter : public DictionaryConverter<ValueType, null_coding> {
+ public:
+  Status AppendValue(PyObject* obj) override {
+    ARROW_ASSIGN_OR_RAISE(string_view_, ValueConverter<ValueType>::FromPython(obj));
+    // DCHECK_GE(string_view_.size, 0);
+    RETURN_NOT_OK(this->typed_builder_->Append(string_view_.bytes,
+                                               static_cast<int32_t>(string_view_.size)));
+    return Status::OK();
+  }
+
+ protected:
+  // Create a single instance of PyBytesView here to prevent unnecessary object
+  // creation/destruction
+  PyBytesView string_view_;
+};
+
+template <NullCoding null_coding>
+class FixedSizeBinaryDictionaryConverter
+    : public DictionaryConverter<FixedSizeBinaryType, null_coding> {
+ public:
+  explicit FixedSizeBinaryDictionaryConverter(int32_t byte_width)
+      : byte_width_(byte_width) {}
+
+  Status AppendValue(PyObject* obj) override {
+    ARROW_ASSIGN_OR_RAISE(
+        string_view_, ValueConverter<FixedSizeBinaryType>::FromPython(obj, byte_width_));
+    RETURN_NOT_OK(this->typed_builder_->Append(string_view_.bytes,

Review comment:
       Same here: it would be good to have `TypedBuilder::Append(string_view)`.

##########
File path: python/pyarrow/tests/test_convert_builtin.py
##########
@@ -1640,3 +1640,85 @@ def test_map_from_tuples():
     for entry in [[(5,)], [()], [('5', 'foo', True)]]:
         with pytest.raises(ValueError, match="(?i)tuple size"):
             pa.array([entry], type=pa.map_('i4', 'i4'))
+
+
+def test_dictionary_from_boolean():
+    typ = pa.dictionary(pa.int8(), value_type=pa.bool_())
+    a = pa.array([False, False, True, False, True], type=typ)
+    assert isinstance(a.type, pa.DictionaryType)
+    assert a.type.equals(typ)
+
+    expected_indices = pa.array([0, 0, 1, 0, 1], type=pa.int8())
+    expected_dictionary = pa.array([False, True], type=pa.bool_())
+    assert a.indices.equals(expected_indices)
+    assert a.dictionary.equals(expected_dictionary)
+
+
+@pytest.mark.parametrize('value_type', [
+    pa.int8(),
+    pa.int16(),
+    pa.int32(),
+    pa.int64(),
+    pa.uint8(),
+    pa.uint16(),
+    pa.uint32(),
+    pa.uint64(),
+    pa.float32(),
+    pa.float64(),
+    pa.date32(),
+    pa.date64(),
+])
+def test_dictionary_from_integers(value_type):
+    typ = pa.dictionary(pa.int8(), value_type=value_type)
+    a = pa.array([1, 2, 1, 1, 2, 3], type=typ)
+    assert isinstance(a.type, pa.DictionaryType)
+    assert a.type.equals(typ)
+
+    expected_indices = pa.array([0, 1, 0, 0, 1, 2], type=pa.int8())

Review comment:
       Can you test with nulls at some point?




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

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



[GitHub] [arrow] kszucs commented on pull request #8008: ARROW-9369: [Python] Support conversion from python sequence to dictionary type

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


   We can close this in favor of #8088


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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #8008: ARROW-9369: [Python] Support conversion from python sequence to dictionary type

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



##########
File path: python/pyarrow/scalar.pxi
##########
@@ -687,6 +687,44 @@ cdef class DictionaryScalar(Scalar):
     Concrete class for dictionary-encoded scalars.
     """
 
+    def __init__(self, index, dictionary, type):

Review comment:
       Maybe we can put this in some `from_..` class method? (to keep it consistent with the other scalars that `__init__` raises)




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

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



[GitHub] [arrow] kszucs commented on a change in pull request #8008: ARROW-9369: [Python] Support conversion from python sequence to dictionary type

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



##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -1123,6 +1168,50 @@ class DecimalConverter : public TypedConverter<arrow::Decimal128Type, null_codin
   std::shared_ptr<DecimalType> decimal_type_;
 };
 
+#define DICTIONARY_PRIMITIVE(TYPE_ENUM, TYPE_CLASS)                 \
+  case Type::TYPE_ENUM:                                             \
+    *out = std::unique_ptr<SeqConverter>(                           \
+        new PrimitiveDictionaryConverter<TYPE_CLASS, null_coding>); \
+    break;
+
+#define DICTIONARY_BINARY_LIKE(TYPE_ENUM, TYPE_CLASS)                \
+  case Type::TYPE_ENUM:                                              \
+    *out = std::unique_ptr<SeqConverter>(                            \
+        new BinaryLikeDictionaryConverter<TYPE_CLASS, null_coding>); \
+    break;
+
+template <NullCoding null_coding>
+Status GetDictionaryConverter(const std::shared_ptr<DataType>& type,
+                              std::unique_ptr<SeqConverter>* out) {
+  const auto& dict_type = checked_cast<const DictionaryType&>(*type);
+  const auto& value_type = dict_type.value_type();
+
+  switch (value_type->id()) {
+    DICTIONARY_PRIMITIVE(BOOL, BooleanType);
+    DICTIONARY_PRIMITIVE(INT8, Int8Type);
+    DICTIONARY_PRIMITIVE(INT16, Int16Type);
+    DICTIONARY_PRIMITIVE(INT32, Int32Type);
+    DICTIONARY_PRIMITIVE(INT64, Int64Type);
+    DICTIONARY_PRIMITIVE(UINT8, UInt8Type);
+    DICTIONARY_PRIMITIVE(UINT16, UInt16Type);
+    DICTIONARY_PRIMITIVE(UINT32, UInt32Type);
+    DICTIONARY_PRIMITIVE(UINT64, UInt64Type);
+    DICTIONARY_PRIMITIVE(HALF_FLOAT, HalfFloatType);
+    DICTIONARY_PRIMITIVE(FLOAT, FloatType);
+    DICTIONARY_PRIMITIVE(DOUBLE, DoubleType);
+    DICTIONARY_PRIMITIVE(DATE32, Date32Type);
+    DICTIONARY_PRIMITIVE(DATE64, Date64Type);
+    DICTIONARY_BINARY_LIKE(BINARY, BinaryType);
+    DICTIONARY_BINARY_LIKE(STRING, StringType);
+    // DICTIONARY_BINARY_LIKE(LARGE_BINARY, LargeBinaryType);
+    // DICTIONARY_BINARY_LIKE(LARGE_STRING, LargeStringType);

Review comment:
       Yes, there are a couple of design decisions we need to make because of the following problems:
   - the index key type is ignored since adaptive builder is used
   - large binary/string types are not supported by the builder which I'm not sure whether is intentional or not




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

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



[GitHub] [arrow] pitrou commented on a change in pull request #8008: ARROW-9369: [Python] Support conversion from python sequence to dictionary type

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



##########
File path: cpp/src/arrow/python/python_to_arrow.cc
##########
@@ -1123,6 +1168,50 @@ class DecimalConverter : public TypedConverter<arrow::Decimal128Type, null_codin
   std::shared_ptr<DecimalType> decimal_type_;
 };
 
+#define DICTIONARY_PRIMITIVE(TYPE_ENUM, TYPE_CLASS)                 \
+  case Type::TYPE_ENUM:                                             \
+    *out = std::unique_ptr<SeqConverter>(                           \
+        new PrimitiveDictionaryConverter<TYPE_CLASS, null_coding>); \
+    break;
+
+#define DICTIONARY_BINARY_LIKE(TYPE_ENUM, TYPE_CLASS)                \
+  case Type::TYPE_ENUM:                                              \
+    *out = std::unique_ptr<SeqConverter>(                            \
+        new BinaryLikeDictionaryConverter<TYPE_CLASS, null_coding>); \
+    break;
+
+template <NullCoding null_coding>
+Status GetDictionaryConverter(const std::shared_ptr<DataType>& type,
+                              std::unique_ptr<SeqConverter>* out) {
+  const auto& dict_type = checked_cast<const DictionaryType&>(*type);
+  const auto& value_type = dict_type.value_type();
+
+  switch (value_type->id()) {
+    DICTIONARY_PRIMITIVE(BOOL, BooleanType);
+    DICTIONARY_PRIMITIVE(INT8, Int8Type);
+    DICTIONARY_PRIMITIVE(INT16, Int16Type);
+    DICTIONARY_PRIMITIVE(INT32, Int32Type);
+    DICTIONARY_PRIMITIVE(INT64, Int64Type);
+    DICTIONARY_PRIMITIVE(UINT8, UInt8Type);
+    DICTIONARY_PRIMITIVE(UINT16, UInt16Type);
+    DICTIONARY_PRIMITIVE(UINT32, UInt32Type);
+    DICTIONARY_PRIMITIVE(UINT64, UInt64Type);
+    DICTIONARY_PRIMITIVE(HALF_FLOAT, HalfFloatType);
+    DICTIONARY_PRIMITIVE(FLOAT, FloatType);
+    DICTIONARY_PRIMITIVE(DOUBLE, DoubleType);
+    DICTIONARY_PRIMITIVE(DATE32, Date32Type);
+    DICTIONARY_PRIMITIVE(DATE64, Date64Type);
+    DICTIONARY_BINARY_LIKE(BINARY, BinaryType);
+    DICTIONARY_BINARY_LIKE(STRING, StringType);
+    // DICTIONARY_BINARY_LIKE(LARGE_BINARY, LargeBinaryType);
+    // DICTIONARY_BINARY_LIKE(LARGE_STRING, LargeStringType);

Review comment:
       Is there a reason this isn't enabled? Is the PR unfinished?




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

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #8008: ARROW-9369: [Python] Support conversion from python sequence to dictionary type

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



##########
File path: python/pyarrow/scalar.pxi
##########
@@ -687,6 +687,44 @@ cdef class DictionaryScalar(Scalar):
     Concrete class for dictionary-encoded scalars.
     """
 
+    def __init__(self, index, dictionary, type):

Review comment:
       (and can you also add some tests for this construction method?)




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

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



[GitHub] [arrow] github-actions[bot] commented on pull request #8008: ARROW-9369: [Python] Support conversion from python sequence to dictionary type

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


   https://issues.apache.org/jira/browse/ARROW-9369


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

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



[GitHub] [arrow] kszucs closed pull request #8008: ARROW-9369: [Python] Support conversion from python sequence to dictionary type

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


   


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