From ca526a708e7fc094c96219dd43fd0c4cc81a16f2 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Wed, 30 Nov 2022 17:04:29 +0100 Subject: [PATCH 01/49] Produce a __dataframe__ object - squshed commits from #14613 --- python/pyarrow/interchange/__init__.py | 16 + python/pyarrow/interchange/buffer.py | 103 ++++ python/pyarrow/interchange/column.py | 514 ++++++++++++++++++ python/pyarrow/interchange/dataframe.py | 190 +++++++ python/pyarrow/interchange/from_dataframe.py | 245 +++++++++ python/pyarrow/table.pxi | 30 + python/pyarrow/tests/interchange/__init__.py | 16 + .../pyarrow/tests/interchange/test_extra.py | 87 +++ .../interchange/test_interchange_spec.py | 178 ++++++ 9 files changed, 1379 insertions(+) create mode 100644 python/pyarrow/interchange/__init__.py create mode 100644 python/pyarrow/interchange/buffer.py create mode 100644 python/pyarrow/interchange/column.py create mode 100644 python/pyarrow/interchange/dataframe.py create mode 100644 python/pyarrow/interchange/from_dataframe.py create mode 100644 python/pyarrow/tests/interchange/__init__.py create mode 100644 python/pyarrow/tests/interchange/test_extra.py create mode 100644 python/pyarrow/tests/interchange/test_interchange_spec.py diff --git a/python/pyarrow/interchange/__init__.py b/python/pyarrow/interchange/__init__.py new file mode 100644 index 00000000000..13a83393a91 --- /dev/null +++ b/python/pyarrow/interchange/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/python/pyarrow/interchange/buffer.py b/python/pyarrow/interchange/buffer.py new file mode 100644 index 00000000000..9f30f2b99e3 --- /dev/null +++ b/python/pyarrow/interchange/buffer.py @@ -0,0 +1,103 @@ +# 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. + +from __future__ import annotations +import enum + +import pyarrow as pa + + +class DlpackDeviceType(enum.IntEnum): + """Integer enum for device type codes matching DLPack.""" + + CPU = 1 + CUDA = 2 + CPU_PINNED = 3 + OPENCL = 4 + VULKAN = 7 + METAL = 8 + VPI = 9 + ROCM = 10 + + +class _PyArrowBuffer: + """ + Data in the buffer is guaranteed to be contiguous in memory. + Note that there is no dtype attribute present, a buffer can be thought of + as simply a block of memory. However, if the column that the buffer is + attached to has a dtype that's supported by DLPack and ``__dlpack__`` is + implemented, then that dtype information will be contained in the return + value from ``__dlpack__``. + This distinction is useful to support both data exchange via DLPack on a + buffer and (b) dtypes like variable-length strings which do not have a + fixed number of bytes per element. + """ + + def __init__(self, x: pa.Buffer, allow_copy: bool = True) -> None: + """ + Handle PyArrow Buffers. + """ + self._x = x + + @property + def bufsize(self) -> int: + """ + Buffer size in bytes. + """ + return self._x.size + + @property + def ptr(self) -> int: + """ + Pointer to start of the buffer as an integer. + """ + return self._x.address + + def __dlpack__(self): + """ + Produce DLPack capsule (see array API standard). + Raises: + - TypeError : if the buffer contains unsupported dtypes. + - NotImplementedError : if DLPack support is not implemented + Useful to have to connect to array libraries. Support optional because + it's not completely trivial to implement for a Python-only library. + """ + raise NotImplementedError("__dlpack__") + + def __dlpack_device__(self) -> tuple[DlpackDeviceType, int | None]: + """ + Device type and device ID for where the data in the buffer resides. + Uses device type codes matching DLPack. + Note: must be implemented even if ``__dlpack__`` is not. + """ + if self._x.is_cpu: + return (DlpackDeviceType.CPU, None) + else: + raise NotImplementedError("__dlpack_device__") + + def __repr__(self) -> str: + return ( + "PyArrowBuffer(" + + str( + { + "bufsize": self.bufsize, + "ptr": self.ptr, + "device": self.__dlpack_device__()[0].name, + } + ) + + ")" + ) diff --git a/python/pyarrow/interchange/column.py b/python/pyarrow/interchange/column.py new file mode 100644 index 00000000000..a5c9d4a2833 --- /dev/null +++ b/python/pyarrow/interchange/column.py @@ -0,0 +1,514 @@ +# 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. + +from __future__ import annotations + +import enum +from typing import ( + Any, + Dict, + Iterable, + Optional, + Tuple, +) + +import sys +if sys.version_info >= (3, 8): + from typing import TypedDict +else: + from typing_extensions import TypedDict + +import pyarrow as pa +import pyarrow.compute as pc +from pyarrow.interchange.buffer import _PyArrowBuffer + + +class DtypeKind(enum.IntEnum): + """ + Integer enum for data types. + Attributes + ---------- + INT : int + Matches to signed integer data type. + UINT : int + Matches to unsigned integer data type. + FLOAT : int + Matches to floating point data type. + BOOL : int + Matches to boolean data type. + STRING : int + Matches to string data type (UTF-8 encoded). + DATETIME : int + Matches to datetime data type. + CATEGORICAL : int + Matches to categorical data type. + """ + + INT = 0 + UINT = 1 + FLOAT = 2 + BOOL = 20 + STRING = 21 # UTF-8 + DATETIME = 22 + CATEGORICAL = 23 + + +Dtype = Tuple[DtypeKind, int, str, str] # see Column.dtype + + +_PYARROW_KINDS = { + pa.int8(): (DtypeKind.INT, "c"), + pa.int16(): (DtypeKind.INT, "s"), + pa.int32(): (DtypeKind.INT, "i"), + pa.int64(): (DtypeKind.INT, "l"), + pa.uint8(): (DtypeKind.UINT, "C"), + pa.uint16(): (DtypeKind.UINT, "S"), + pa.uint32(): (DtypeKind.UINT, "I"), + pa.uint64(): (DtypeKind.UINT, "L"), + pa.float16(): (DtypeKind.FLOAT, "e"), + pa.float32(): (DtypeKind.FLOAT, "f"), + pa.float64(): (DtypeKind.FLOAT, "g"), + pa.bool_(): (DtypeKind.BOOL, "b"), + pa.string(): (DtypeKind.STRING, "u"), # utf-8 + pa.large_string(): (DtypeKind.STRING, "U"), +} + + +class ColumnNullType(enum.IntEnum): + """ + Integer enum for null type representation. + Attributes + ---------- + NON_NULLABLE : int + Non-nullable column. + USE_NAN : int + Use explicit float NaN value. + USE_SENTINEL : int + Sentinel value besides NaN. + USE_BITMASK : int + The bit is set/unset representing a null on a certain position. + USE_BYTEMASK : int + The byte is set/unset representing a null on a certain position. + """ + + NON_NULLABLE = 0 + USE_NAN = 1 + USE_SENTINEL = 2 + USE_BITMASK = 3 + USE_BYTEMASK = 4 + + +class ColumnBuffers(TypedDict): + # first element is a buffer containing the column data; + # second element is the data buffer's associated dtype + data: Tuple[_PyArrowBuffer, Dtype] + + # first element is a buffer containing mask values indicating missing data; + # second element is the mask value buffer's associated dtype. + # None if the null representation is not a bit or byte mask + validity: Optional[Tuple[_PyArrowBuffer, Dtype]] + + # first element is a buffer containing the offset values for + # variable-size binary data (e.g., variable-length strings); + # second element is the offsets buffer's associated dtype. + # None if the data buffer does not have an associated offsets buffer + offsets: Optional[Tuple[_PyArrowBuffer, Dtype]] + + +class CategoricalDescription(TypedDict): + # whether the ordering of dictionary indices is semantically meaningful + is_ordered: bool + # whether a dictionary-style mapping of categorical values to other objects + # exists + is_dictionary: bool + # Python-level only (e.g. ``{int: str}``). + # None if not a dictionary-style categorical. + # categories: Optional[Column] + + +class Endianness: + """Enum indicating the byte-order of a data-type.""" + + LITTLE = "<" + BIG = ">" + NATIVE = "=" + NA = "|" + + +class NoBufferPresent(Exception): + """Exception to signal that there is no requested buffer.""" + + +class _PyArrowColumn: + """ + A column object, with only the methods and properties required by the + interchange protocol defined. + A column can contain one or more chunks. Each chunk can contain up to three + buffers - a data buffer, a mask buffer (depending on null representation), + and an offsets buffer (if variable-size binary; e.g., variable-length + strings). + TBD: Arrow has a separate "null" dtype, and has no separate mask concept. + Instead, it seems to use "children" for both columns with a bit mask, + and for nested dtypes. Unclear whether this is elegant or confusing. + This design requires checking the null representation explicitly. + The Arrow design requires checking: + 1. the ARROW_FLAG_NULLABLE (for sentinel values) + 2. if a column has two children, combined with one of those children + having a null dtype. + Making the mask concept explicit seems useful. One null dtype would + not be enough to cover both bit and byte masks, so that would mean + even more checking if we did it the Arrow way. + TBD: there's also the "chunk" concept here, which is implicit in Arrow as + multiple buffers per array (= column here). Semantically it may make + sense to have both: chunks were meant for example for lazy evaluation + of data which doesn't fit in memory, while multiple buffers per column + could also come from doing a selection operation on a single + contiguous buffer. + Given these concepts, one would expect chunks to be all of the same + size (say a 10,000 row dataframe could have 10 chunks of 1,000 rows), + while multiple buffers could have data-dependent lengths. Not an issue + in pandas if one column is backed by a single NumPy array, but in + Arrow it seems possible. + Are multiple chunks *and* multiple buffers per column necessary for + the purposes of this interchange protocol, or must producers either + reuse the chunk concept for this or copy the data? + Note: this Column object can only be produced by ``__dataframe__``, so + doesn't need its own version or ``__column__`` protocol. + """ + + def __init__( + self, column: pa.Array | pa.ChunkedArray, allow_copy: bool = True + ) -> None: + """ + Handles PyArrow Arrays and ChunkedArrays. + """ + # Store the column as a private attribute + self._col = column + self._allow_copy = allow_copy + + @property + def size(self) -> int: + """ + Size of the column, in elements. + Corresponds to DataFrame.num_rows() if column is a single chunk; + equal to size of this current chunk otherwise. + Is a method rather than a property because it may cause a (potentially + expensive) computation for some dataframe implementations. + """ + return len(self._col) + + @property + def offset(self) -> int: + """ + Offset of first element. + May be > 0 if using chunks; for example for a column with N chunks of + equal size M (only the last chunk may be shorter), + ``offset = n * M``, ``n = 0 .. N-1``. + """ + if isinstance(self._col, pa.Array): + return self._col.offset + else: + # ChunkedArray gets copied with `combine_chunks` so the offset will + # always be 0 + return 0 + + @property + def dtype(self) -> Tuple[DtypeKind, int, str, str]: + """ + Dtype description as a tuple ``(kind, bit-width, format string, + endianness)``. + Bit-width : the number of bits as an integer + Format string : data type description format string in Apache Arrow C + Data Interface format. + Endianness : current only native endianness (``=``) is supported + Notes: + - Kind specifiers are aligned with DLPack where possible (hence the + jump to 20, leave enough room for future extension) + - Masks must be specified as boolean with either bit width 1 (for + bit masks) or 8 (for byte masks). + - Dtype width in bits was preferred over bytes + - Endianness isn't too useful, but included now in case in the + future we need to support non-native endianness + - Went with Apache Arrow format strings over NumPy format strings + because they're more complete from a dataframe perspective + - Format strings are mostly useful for datetime specification, and + for categoricals. + - For categoricals, the format string describes the type of the + categorical in the data buffer. In case of a separate encoding of + the categorical (e.g. an integer to string mapping), this can + be derived from ``self.describe_categorical``. + - Data types not included: complex, Arrow-style null, binary, + decimal, and nested (list, struct, map, union) dtypes. + """ + dtype = self._col.type + try: + bit_width = dtype.bit_width + except ValueError: # in case of a variable-length strings + bit_width = 8 + + if pa.types.is_timestamp(dtype): + kind = DtypeKind.DATETIME + ts = dtype.unit[0] + tz = dtype.tz if dtype.tz else "" + f_string = "ts{ts}:{tz}".format(ts=ts, tz=tz) + return kind, bit_width, f_string, Endianness.NATIVE + elif pa.types.is_dictionary(dtype): + kind = DtypeKind.CATEGORICAL + f_string = "L" + return kind, bit_width, f_string, Endianness.NATIVE + else: + return self._dtype_from_arrowdtype(dtype, bit_width) + + def _dtype_from_arrowdtype( + self, dtype, bit_width + ) -> Tuple[DtypeKind, int, str, str]: + """ + See `self.dtype` for details. + """ + # Note: 'c' (complex) not handled yet (not in array spec v1). + # 'b', 'B' (bytes), 'S', 'a', (old-style string) 'V' (void) + # not handled datetime and timedelta both map to datetime + # (is timedelta handled?) + + kind, f_string = _PYARROW_KINDS.get(dtype, (None, None)) + if kind is None: + raise ValueError( + f"Data type {dtype} not supported by interchange protocol") + + return kind, bit_width, f_string, Endianness.NATIVE + + @property + def describe_categorical(self) -> CategoricalDescription: + """ + If the dtype is categorical, there are two options: + - There are only values in the data buffer. + - There is a separate non-categorical Column encoding categorical + values. + Raises TypeError if the dtype is not categorical + Returns the dictionary with description on how to interpret the + data buffer: + - "is_ordered" : bool, whether the ordering of dictionary indices + is semantically meaningful. + - "is_dictionary" : bool, whether a mapping of + categorical values to other objects exists + - "categories" : Column representing the (implicit) mapping of + indices to category values (e.g. an array of + cat1, cat2, ...). None if not a dictionary-style + categorical. + TBD: are there any other in-memory representations that are needed? + """ + if isinstance(self._col, pa.ChunkedArray): + arr = self._col.combine_chunks() + else: + arr = self._col + + if not pa.types.is_dictionary(arr.type): + raise TypeError( + "describe_categorical only works on a column with " + "categorical dtype!" + ) + + return { + "is_ordered": self._col.type.ordered, + "is_dictionary": True, + "categories": _PyArrowColumn(arr.dictionary), + } + + @property + def describe_null(self) -> Tuple[ColumnNullType, Any]: + """ + Return the missing value (or "null") representation the column dtype + uses, as a tuple ``(kind, value)``. + Value : if kind is "sentinel value", the actual value. If kind is a bit + mask or a byte mask, the value (0 or 1) indicating a missing value. + None otherwise. + """ + # In case of no missing values, we need to set ColumnNullType to + # non nullable as in the current __dataframe__ protocol bit/byte masks + # can not be None + if self.null_count == 0: + return ColumnNullType.NON_NULLABLE, None + else: + return ColumnNullType.USE_BITMASK, 0 + + @property + def null_count(self) -> int: + """ + Number of null elements, if known. + Note: Arrow uses -1 to indicate "unknown", but None seems cleaner. + """ + return self._col.null_count + + @property + def metadata(self) -> Dict[str, Any]: + """ + The metadata for the column. See `DataFrame.metadata` for more details. + """ + pass + + def num_chunks(self) -> int: + """ + Return the number of chunks the column consists of. + """ + if isinstance(self._col, pa.Array): + n_chunks = 1 + else: + n_chunks = self._col.num_chunks + return n_chunks + + def get_chunks( + self, n_chunks: Optional[int] = None + ) -> Iterable[_PyArrowColumn]: + """ + Return an iterator yielding the chunks. + See `DataFrame.get_chunks` for details on ``n_chunks``. + """ + if n_chunks and n_chunks > 1: + chunk_size = self.size // n_chunks + if self.size % n_chunks != 0: + chunk_size += 1 + + if isinstance(self._col, pa.ChunkedArray): + array = self._col.combine_chunks() + else: + array = self._col + + i = 0 + for start in range(0, chunk_size * n_chunks, chunk_size): + yield _PyArrowColumn( + array.slice(start, chunk_size), self._allow_copy + ) + i += 1 + + elif isinstance(self._col, pa.ChunkedArray): + return [ + _PyArrowColumn(chunk, self._allow_copy) + for chunk in self._col.chunks + ] + else: + yield self + + def get_buffers(self) -> ColumnBuffers: + """ + Return a dictionary containing the underlying buffers. + The returned dictionary has the following contents: + - "data": a two-element tuple whose first element is a buffer + containing the data and whose second element is the data + buffer's associated dtype. + - "validity": a two-element tuple whose first element is a buffer + containing mask values indicating missing data and + whose second element is the mask value buffer's + associated dtype. None if the null representation is + not a bit or byte mask. + - "offsets": a two-element tuple whose first element is a buffer + containing the offset values for variable-size binary + data (e.g., variable-length strings) and whose second + element is the offsets buffer's associated dtype. None + if the data buffer does not have an associated offsets + buffer. + """ + buffers: ColumnBuffers = { + "data": self._get_data_buffer(), + "validity": None, + "offsets": None, + } + + try: + buffers["validity"] = self._get_validity_buffer() + except NoBufferPresent: + pass + + try: + buffers["offsets"] = self._get_offsets_buffer() + except NoBufferPresent: + pass + + return buffers + + def _get_data_buffer( + self, + ) -> Tuple[_PyArrowBuffer, Any]: # Any is for self.dtype tuple + """ + Return the buffer containing the data and the buffer's + associated dtype. + """ + if isinstance(self._col, pa.ChunkedArray): + array = self._col.combine_chunks() + else: + array = self._col + dtype = self.dtype + + # In case of boolean arrays, cast to uint8 array + # as bit packed buffers are not supported + if pa.types.is_boolean(array.type): + array = pc.cast(array, pa.uint8()) + dtype = _PyArrowColumn(array).dtype + # In case of dictionary arrays, use indices + # to define a buffer, codes are transferred through + # describe_categorical() + if pa.types.is_dictionary(array.type): + array = array.indices + dtype = _PyArrowColumn(array).dtype + + n = len(array.buffers()) + if n == 2: + return _PyArrowBuffer(array.buffers()[1]), dtype + elif n == 3: + return _PyArrowBuffer(array.buffers()[2]), dtype + + def _get_validity_buffer(self) -> Tuple[_PyArrowBuffer, Any]: + """ + Return the buffer containing the mask values indicating missing data + and the buffer's associated dtype. + Raises NoBufferPresent if null representation is not a bit or byte + mask. + """ + # Define the dtype of the returned buffer + dtype = (DtypeKind.BOOL, 1, "b", Endianness.NATIVE) + if isinstance(self._col, pa.ChunkedArray): + array = self._col.combine_chunks() + else: + array = self._col + buff = array.buffers()[0] + if buff: + return _PyArrowBuffer(buff), dtype + else: + raise NoBufferPresent( + "There are no missing values so " + "does not have a separate mask") + + def _get_offsets_buffer(self) -> Tuple[_PyArrowBuffer, Any]: + """ + Return the buffer containing the offset values for variable-size binary + data (e.g., variable-length strings) and the buffer's associated dtype. + Raises NoBufferPresent if the data buffer does not have an associated + offsets buffer. + """ + if isinstance(self._col, pa.ChunkedArray): + array = self._col.combine_chunks() + else: + array = self._col + n = len(array.buffers()) + if n == 2: + raise NoBufferPresent( + "This column has a fixed-length dtype so " + "it does not have an offsets buffer" + ) + elif n == 3: + # Define the dtype of the returned buffer + dtype = (DtypeKind.INT, 32, "i", Endianness.NATIVE) + return _PyArrowBuffer(array.buffers()[1]), dtype diff --git a/python/pyarrow/interchange/dataframe.py b/python/pyarrow/interchange/dataframe.py new file mode 100644 index 00000000000..278ee102ec9 --- /dev/null +++ b/python/pyarrow/interchange/dataframe.py @@ -0,0 +1,190 @@ +# 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. + +from __future__ import annotations +from typing import ( + Any, + Iterable, + Optional, + Sequence, +) + +import pyarrow as pa + +from pyarrow.interchange.column import _PyArrowColumn + + +class _PyArrowDataFrame: + """ + A data frame class, with only the methods required by the interchange + protocol defined. + A "data frame" represents an ordered collection of named columns. + A column's "name" must be a unique string. + Columns may be accessed by name or by position. + This could be a public data frame class, or an object with the methods and + attributes defined on this DataFrame class could be returned from the + ``__dataframe__`` method of a public data frame class in a library adhering + to the dataframe interchange protocol specification. + """ + + def __init__( + self, df: pa.Table, nan_as_null: bool = False, allow_copy: bool = True + ) -> None: + """ + Constructor - an instance of this (private) class is returned from + `pa.Table.__dataframe__`. + """ + self._df = df + # ``nan_as_null`` is a keyword intended for the consumer to tell the + # producer to overwrite null values in the data with ``NaN`` (or + # ``NaT``). This currently has no effect; once support for nullable + # extension dtypes is added, this value should be propagated to + # columns. + self._nan_as_null = nan_as_null + self._allow_copy = allow_copy + + def __dataframe__( + self, nan_as_null: bool = False, allow_copy: bool = True + ) -> _PyArrowDataFrame: + """ + Construct a new exchange object, potentially changing the parameters. + ``nan_as_null`` is a keyword intended for the consumer to tell the + producer to overwrite null values in the data with ``NaN``. + It is intended for cases where the consumer does not support the bit + mask or byte mask that is the producer's native representation. + ``allow_copy`` is a keyword that defines whether or not the library is + allowed to make a copy of the data. For example, copying data would be + necessary if a library supports strided buffers, given that this + protocol specifies contiguous buffers. + """ + return _PyArrowDataFrame(self._df, nan_as_null, allow_copy) + + @property + def metadata(self) -> dict[str, Any]: + """ + The metadata for the data frame, as a dictionary with string keys. The + contents of `metadata` may be anything, they are meant for a library + to store information that it needs to, e.g., roundtrip losslessly or + for two implementations to share data that is not (yet) part of the + interchange protocol specification. For avoiding collisions with other + entries, please add name the keys with the name of the library + followed by a period and the desired name, e.g, ``pandas.indexcol``. + """ + # The metadata for the data frame, as a dictionary with string keys. + # Add schema metadata here (pandas metadata or custom metadata) + if self._df.schema.metadata: + schema_metadata = {k.decode('utf8'): v.decode('utf8') + for k, v in self._df.schema.metadata.items()} + return schema_metadata + else: + return {} + + def num_columns(self) -> int: + """ + Return the number of columns in the DataFrame. + """ + return self._df.num_columns + + def num_rows(self) -> int: + """ + Return the number of rows in the DataFrame, if available. + """ + return self._df.num_rows + + def num_chunks(self) -> int: + """ + Return the number of chunks the DataFrame consists of. + """ + return self._df.column(0).num_chunks + + def column_names(self) -> Iterable[str]: + """ + Return an iterator yielding the column names. + """ + return self._df.column_names + + def get_column(self, i: int) -> _PyArrowColumn: + """ + Return the column at the indicated position. + """ + return _PyArrowColumn(self._df.column(i), + allow_copy=self._allow_copy) + + def get_column_by_name(self, name: str) -> _PyArrowColumn: + """ + Return the column whose name is the indicated name. + """ + return _PyArrowColumn(self._df.column(name), + allow_copy=self._allow_copy) + + def get_columns(self) -> Iterable[_PyArrowColumn]: + """ + Return an iterator yielding the columns. + """ + return [ + _PyArrowColumn(col, allow_copy=self._allow_copy) + for col in self._df.columns + ] + + def select_columns(self, indices: Sequence[int]) -> _PyArrowDataFrame: + """ + Create a new DataFrame by selecting a subset of columns by index. + """ + return _PyArrowDataFrame( + self._df.select(list(indices)), self._nan_as_null, self._allow_copy + ) + + def select_columns_by_name( + self, names: Sequence[str] + ) -> _PyArrowDataFrame: + """ + Create a new DataFrame by selecting a subset of columns by name. + """ + return _PyArrowDataFrame( + self._df.select(list(names)), self._nan_as_null, self._allow_copy + ) + + def get_chunks( + self, n_chunks: Optional[int] = None + ) -> Iterable[_PyArrowDataFrame]: + """ + Return an iterator yielding the chunks. + By default (None), yields the chunks that the data is stored as by the + producer. If given, ``n_chunks`` must be a multiple of + ``self.num_chunks()``, meaning the producer must subdivide each chunk + before yielding it. + Note that the producer must ensure that all columns are chunked the + same way. + """ + if n_chunks and n_chunks > 1: + chunk_size = self.num_rows() // n_chunks + if self.num_rows() % n_chunks != 0: + chunk_size += 1 + batches = self._df.to_batches(max_chunksize=chunk_size) + # In case when the size of the chunk is such that the resulting + # list is one less chunk then n_chunks -> append an empty chunk + if len(batches) == n_chunks - 1: + batches.append(pa.record_batch([[]], schema=self._df.schema)) + else: + batches = self._df.to_batches() + + iterator_tables = [_PyArrowDataFrame( + pa.Table.from_batches([batch]), self._nan_as_null, self._allow_copy + ) + for batch in batches + ] + return iterator_tables diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py new file mode 100644 index 00000000000..b37f1d06960 --- /dev/null +++ b/python/pyarrow/interchange/from_dataframe.py @@ -0,0 +1,245 @@ +# 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. + +from __future__ import annotations + +from typing import ( + Any, +) + +from pyarrow.interchange.buffer import _PyArrowBuffer +from pyarrow.interchange.column import ( + _PyArrowColumn, + DtypeKind, +) +from pyarrow.interchange.dataframe import _PyArrowDataFrame + +import numpy as np +import pyarrow as pa + + +# A typing protocol could be added later to let Mypy validate code using +# `from_dataframe` better. +DataFrameObject = Any +ColumnObject = Any +BufferObject = Any + + +def from_dataframe(df: DataFrameObject, allow_copy=True) -> pa.Table: + """ + Build a ``pa.Table`` from any DataFrame supporting the interchange + protocol. + + Parameters + ---------- + df : DataFrameObject + Object supporting the interchange protocol, i.e. `__dataframe__` + method. + allow_copy : bool, default: True + Whether to allow copying the memory to perform the conversion + (if false then zero-copy approach is requested). + Returns + ------- + pa.Table + """ + if isinstance(df, pa.Table): + return df + + if not hasattr(df, "__dataframe__"): + raise ValueError("`df` does not support __dataframe__") + + return _from_dataframe(df.__dataframe__(allow_copy=allow_copy)) + + +def _from_dataframe(df: DataFrameObject, allow_copy=True): + """ + Build a ``pa.Table`` from the DataFrame interchange object. + Parameters + ---------- + df : DataFrameObject + Object supporting the interchange protocol, i.e. `__dataframe__` + method. + allow_copy : bool, default: True + Whether to allow copying the memory to perform the conversion + (if false then zero-copy approach is requested). + Returns + ------- + pa.Table + """ + pass + + +def protocol_df_chunk_to_pyarrow(df: DataFrameObject) -> pa.Table: + """ + Convert interchange protocol chunk to ``pd.DataFrame``. + Parameters + ---------- + df : DataFrameObject + Returns + ------- + pa.Table + """ + # We need a dict of columns here, with each column being a NumPy array + # (at least for now, deal with non-NumPy dtypes later). + columns: dict[str, Any] = {} + buffers = [] # hold on to buffers, keeps memory alive + for name in df.column_names(): + if not isinstance(name, str): + raise ValueError(f"Column {name} is not a string") + if name in columns: + raise ValueError(f"Column {name} is not unique") + col = df.get_column_by_name(name) + dtype = col.dtype[0] + if dtype in ( + DtypeKind.INT, + DtypeKind.UINT, + DtypeKind.FLOAT, + DtypeKind.BOOL, + ): + columns[name], buf = primitive_column_to_ndarray(col) + elif dtype == DtypeKind.CATEGORICAL: + columns[name], buf = categorical_column_to_dictionary(col) + elif dtype == DtypeKind.STRING: + columns[name], buf = string_column_to_ndarray(col) + elif dtype == DtypeKind.DATETIME: + columns[name], buf = datetime_column_to_ndarray(col) + else: + raise NotImplementedError(f"Data type {dtype} not handled yet") + + buffers.append(buf) + + pass + + +def primitive_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: + """ + Convert a column holding one of the primitive dtypes to a PyArrow array. + A primitive type is one of: int, uint, float, bool. + Parameters + ---------- + col : ColumnObject + Returns + ------- + tuple + Tuple of pa.Array holding the data and the memory owner object + that keeps the memory alive. + """ + pass + + +def categorical_column_to_dictionary( + col: ColumnObject +) -> tuple[pa.Array, Any]: + """ + Convert a column holding categorical data to a pandas Series. + Parameters + ---------- + col : ColumnObject + Returns + ------- + tuple + Tuple of pa.Array holding the data and the memory owner object + that keeps the memory alive. + """ + pass + + +def string_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: + """ + Convert a column holding string data to a NumPy array. + Parameters + ---------- + col : ColumnObject + Returns + ------- + tuple + Tuple of pa.Array holding the data and the memory owner object + that keeps the memory alive. + """ + pass + + +def parse_datetime_format_str(format_str, data): + """Parse datetime `format_str` to interpret the `data`.""" + pass + + +def datetime_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: + """ + Convert a column holding DateTime data to a NumPy array. + Parameters + ---------- + col : ColumnObject + Returns + ------- + tuple + Tuple of pa.Array holding the data and the memory owner object + that keeps the memory alive. + """ + pass + + +def buffer_to_array( + buffer: BufferObject, + dtype: tuple[DtypeKind, int, str, str], + offset: int = 0, + length: int | None = None, +) -> pa.Array: + """ + Build a NumPy array from the passed buffer. + Parameters + ---------- + buffer : BufferObject + Buffer to build a PyArrow array from. + dtype : tuple + Data type of the buffer conforming protocol dtypes format. + offset : int, default: 0 + Number of elements to offset from the start of the buffer. + length : int, optional + If the buffer is a bit-mask, specifies a number of bits to read + from the buffer. Has no effect otherwise. + Returns + ------- + pa.Array + + Notes + ----- + The returned array doesn't own the memory. The caller of this function + is responsible for keeping the memory owner object alive as long as + the returned NumPy array is being used. + """ + pass + + +def bitmask_to_bool_array( + bitmask: np.ndarray, mask_length: int, first_byte_offset: int = 0 +) -> pa.Array: + """ + Convert bit-mask to a boolean NumPy array. + Parameters + ---------- + bitmask : np.ndarray[uint8] + NumPy array of uint8 dtype representing the bitmask. + mask_length : int + Number of elements in the mask to interpret. + first_byte_offset : int, default: 0 + Number of elements to offset from the start of the first byte. + Returns + ------- + pa.Array[bool] + """ + pass \ No newline at end of file diff --git a/python/pyarrow/table.pxi b/python/pyarrow/table.pxi index c97ad027dd7..35492d4f646 100644 --- a/python/pyarrow/table.pxi +++ b/python/pyarrow/table.pxi @@ -2861,6 +2861,36 @@ cdef class Table(_PandasConvertible): return self.column(key) + # ---------------------------------------------------------------------- + def __dataframe__(self, nan_as_null: bool = False, allow_copy: bool = True): + """ + Return the dataframe interchange object implementing the interchange protocol. + Parameters + ---------- + nan_as_null : bool, default False + Whether to tell the DataFrame to overwrite null values in the data + with ``NaN`` (or ``NaT``). + allow_copy : bool, default True + Whether to allow memory copying when exporting. If set to False + it would cause non-zero-copy exports to fail. + Returns + ------- + DataFrame interchange object + The object which consuming library can use to ingress the dataframe. + Notes + ----- + Details on the interchange protocol: + https://data-apis.org/dataframe-protocol/latest/index.html + `nan_as_null` currently has no effect; once support for nullable extension + dtypes is added, this value should be propagated to columns. + """ + + from pyarrow.interchange.dataframe import _PyArrowDataFrame + + return _PyArrowDataFrame(self, nan_as_null, allow_copy) + + # ---------------------------------------------------------------------- + def slice(self, offset=0, length=None): """ Compute zero-copy slice of this Table. diff --git a/python/pyarrow/tests/interchange/__init__.py b/python/pyarrow/tests/interchange/__init__.py new file mode 100644 index 00000000000..13a83393a91 --- /dev/null +++ b/python/pyarrow/tests/interchange/__init__.py @@ -0,0 +1,16 @@ +# 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. diff --git a/python/pyarrow/tests/interchange/test_extra.py b/python/pyarrow/tests/interchange/test_extra.py new file mode 100644 index 00000000000..4181b117be6 --- /dev/null +++ b/python/pyarrow/tests/interchange/test_extra.py @@ -0,0 +1,87 @@ +# 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. + +from datetime import datetime as dt +import pyarrow as pa +from pyarrow.vendored.version import Version +import pytest + +from pyarrow.interchange.column import ( + _PyArrowColumn, + ColumnNullType, + DtypeKind, +) + +try: + import pandas as pd + import pandas.testing as tm + from pandas.core.interchange.from_dataframe import from_dataframe +except ImportError: + pass + + +def test_datetime(): + df = pd.DataFrame({"A": [dt(2007, 7, 13), None]}) + table = pa.table(df) + col = table.__dataframe__().get_column_by_name("A") + + assert col.size == 2 + assert col.null_count == 1 + assert col.dtype[0] == DtypeKind.DATETIME + assert col.describe_null == (ColumnNullType.USE_BITMASK, 0) + + +@pytest.mark.parametrize( + ["test_data", "kind"], + [ + (["foo", "bar"], 21), + ([1.5, 2.5, 3.5], 2), + ([1, 2, 3, 4], 0), + ], +) +def test_array_to_pyarrowcolumn(test_data, kind): + arr = pa.array(test_data) + arr_column = _PyArrowColumn(arr) + + assert arr_column._col == arr + assert arr_column.size == len(test_data) + assert arr_column.dtype[0] == kind + assert arr_column.num_chunks() == 1 + assert arr_column.null_count == 0 + assert arr_column.get_buffers()["validity"] is None + assert len(list(arr_column.get_chunks())) == 1 + + for chunk in arr_column.get_chunks(): + assert chunk == arr_column + + +@pytest.mark.pandas +def test_offset_of_sliced_array(): + if Version(pd.__version__) < Version("1.5.0"): + pytest.skip("__dataframe__ added to pandas in 1.5.0") + + arr = pa.array([1, 2, 3, 4]) + arr_sliced = arr.slice(2, 2) + + table = pa.table([arr], names=["arr"]) + table_sliced = pa.table([arr_sliced], names=["arr_sliced"]) + + df = from_dataframe(table) + df_sliced = from_dataframe(table_sliced) + + tm.assert_series_equal(df["arr"][2:4], df_sliced["arr_sliced"], + check_index=False, check_names=False) diff --git a/python/pyarrow/tests/interchange/test_interchange_spec.py b/python/pyarrow/tests/interchange/test_interchange_spec.py new file mode 100644 index 00000000000..425e8f7f95d --- /dev/null +++ b/python/pyarrow/tests/interchange/test_interchange_spec.py @@ -0,0 +1,178 @@ +# 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. + +import ctypes + +import pyarrow as pa +import pytest + + +@pytest.mark.parametrize( + "test_data", + [ + {"a": ["foo", "bar"], "b": ["baz", "qux"]}, + {"a": [1.5, 2.5, 3.5], "b": [9.2, 10.5, 11.8]}, + {"A": [1, 2, 3, 4], "B": [1, 2, 3, 4]}, + ], +) +def test_only_one_dtype(test_data): + columns = list(test_data.keys()) + table = pa.table(test_data) + df = table.__dataframe__() + + column_size = len(test_data[columns[0]]) + for column in columns: + null_count = df.get_column_by_name(column).null_count + assert null_count == 0 + assert isinstance(null_count, int) + assert df.get_column_by_name(column).size == column_size + assert df.get_column_by_name(column).offset == 0 + + +def test_mixed_dtypes(): + table = pa.table( + { + "a": [1, 2, 3], # dtype kind INT = 0 + "b": [3, 4, 5], # dtype kind INT = 0 + "c": [1.5, 2.5, 3.5], # dtype kind FLOAT = 2 + "d": [9, 10, 11], # dtype kind INT = 0 + "e": [True, False, True], # dtype kind BOOLEAN = 20 + "f": ["a", "", "c"], # dtype kind STRING = 21 + } + ) + df = table.__dataframe__() + # for meanings of dtype[0] see the spec; we cannot import the + # spec here as this file is expected to be vendored *anywhere*; + # values for dtype[0] are explained above + columns = {"a": 0, "b": 0, "c": 2, "d": 0, "e": 20, "f": 21} + + for column, kind in columns.items(): + col = df.get_column_by_name(column) + assert col.null_count == 0 + assert isinstance(col.null_count, int) + assert col.size == 3 + assert col.offset == 0 + + assert col.dtype[0] == kind + + assert df.get_column_by_name("c").dtype[1] == 64 + + +def test_na_float(): + table = pa.table({"a": [1.0, None, 2.0]}) + df = table.__dataframe__() + col = df.get_column_by_name("a") + assert col.null_count == 1 + assert isinstance(col.null_count, int) + + +def test_noncategorical(): + table = pa.table({"a": [1, 2, 3]}) + df = table.__dataframe__() + col = df.get_column_by_name("a") + with pytest.raises(TypeError, match=".*categorical.*"): + col.describe_categorical + + +def test_categorical(): + import pyarrow as pa + arr = ["Mon", "Tue", "Mon", "Wed", "Mon", "Thu", "Fri", "Sat", "Sun"] + table = pa.table( + {"weekday": pa.array(arr).dictionary_encode()} + ) + + col = table.__dataframe__().get_column_by_name("weekday") + categorical = col.describe_categorical + assert isinstance(categorical["is_ordered"], bool) + assert isinstance(categorical["is_dictionary"], bool) + + +def test_dataframe(): + n = pa.chunked_array([[2, 2, 4], [4, 5, 100]]) + a = pa.chunked_array([["Flamingo", "Parrot", "Cow"], + ["Horse", "Brittle stars", "Centipede"]]) + table = pa.table([n, a], names=['n_legs', 'animals']) + df = table.__dataframe__() + + assert df.num_columns() == 2 + assert df.num_rows() == 6 + assert df.num_chunks() == 2 + assert list(df.column_names()) == ['n_legs', 'animals'] + assert list(df.select_columns((1,)).column_names()) == list( + df.select_columns_by_name(("animals",)).column_names() + ) + + +@pytest.mark.parametrize(["size", "n_chunks"], [(10, 3), (12, 3), (12, 5)]) +def test_df_get_chunks(size, n_chunks): + table = pa.table({"x": list(range(size))}) + df = table.__dataframe__() + chunks = list(df.get_chunks(n_chunks)) + assert len(chunks) == n_chunks + assert sum(chunk.num_rows() for chunk in chunks) == size + + +@pytest.mark.parametrize(["size", "n_chunks"], [(10, 3), (12, 3), (12, 5)]) +def test_column_get_chunks(size, n_chunks): + table = pa.table({"x": list(range(size))}) + df = table.__dataframe__() + chunks = list(df.get_column(0).get_chunks(n_chunks)) + assert len(chunks) == n_chunks + assert sum(chunk.size for chunk in chunks) == size + + +def test_get_columns(): + table = pa.table({"a": [0, 1], "b": [2.5, 3.5]}) + df = table.__dataframe__() + for col in df.get_columns(): + assert col.size == 2 + assert col.num_chunks() == 1 + # for meanings of dtype[0] see the spec; we cannot import the + # spec here as this file is expected to be vendored *anywhere* + assert df.get_column(0).dtype[0] == 0 # INT + assert df.get_column(1).dtype[0] == 2 # FLOAT + + +def test_buffer(): + arr = [0, 1, -1] + table = pa.table({"a": arr}) + df = table.__dataframe__() + col = df.get_column(0) + buf = col.get_buffers() + + dataBuf, dataDtype = buf["data"] + + assert dataBuf.bufsize > 0 + assert dataBuf.ptr != 0 + device, _ = dataBuf.__dlpack_device__() + + # for meanings of dtype[0] see the spec; we cannot import the spec + # here as this file is expected to be vendored *anywhere* + assert dataDtype[0] == 0 # INT + + if device == 1: # CPU-only as we're going to directly read memory here + bitwidth = dataDtype[1] + ctype = { + 8: ctypes.c_int8, + 16: ctypes.c_int16, + 32: ctypes.c_int32, + 64: ctypes.c_int64, + }[bitwidth] + + for idx, truth in enumerate(arr): + val = ctype.from_address(dataBuf.ptr + idx * (bitwidth // 8)).value + assert val == truth, f"Buffer at index {idx} mismatch" From d0ca2b1054a450e9914e4a78c9f03e18b2c6ed30 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Thu, 1 Dec 2022 13:53:33 +0100 Subject: [PATCH 02/49] Add column convert methods --- python/pyarrow/interchange/from_dataframe.py | 390 ++++++++++++++++--- 1 file changed, 339 insertions(+), 51 deletions(-) diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index b37f1d06960..dace516d79e 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -21,15 +21,18 @@ Any, ) -from pyarrow.interchange.buffer import _PyArrowBuffer from pyarrow.interchange.column import ( - _PyArrowColumn, DtypeKind, + ColumnBuffers, + ColumnNullType, ) -from pyarrow.interchange.dataframe import _PyArrowDataFrame +import ctypes import numpy as np import pyarrow as pa +import re + +from pyarrow.interchange.column import Dtype # A typing protocol could be added later to let Mypy validate code using @@ -39,6 +42,24 @@ BufferObject = Any +_PYARROW_DTYPES: dict[DtypeKind, dict[int, Any]] = { + DtypeKind.INT: {8: pa.int8(), + 16: pa.int16(), + 32: pa.int32(), + 64: pa.int64()}, + DtypeKind.UINT: {8: pa.uint8(), + 16: pa.uint16(), + 32: pa.uint32(), + 64: pa.uint64()}, + DtypeKind.FLOAT: {16: pa.float16(), + 32: pa.float32(), + 64: pa.float64()}, + DtypeKind.BOOL: {1: pa.bool_()}, + DtypeKind.STRING: {1: pa.string(), + 8: pa.large_string()}, +} + + def from_dataframe(df: DataFrameObject, allow_copy=True) -> pa.Table: """ Build a ``pa.Table`` from any DataFrame supporting the interchange @@ -108,15 +129,15 @@ def protocol_df_chunk_to_pyarrow(df: DataFrameObject) -> pa.Table: DtypeKind.INT, DtypeKind.UINT, DtypeKind.FLOAT, - DtypeKind.BOOL, + DtypeKind.STRING, ): - columns[name], buf = primitive_column_to_ndarray(col) + columns[name], buf = column_to_array(col) + elif dtype == DtypeKind.BOOL: + columns[name], buf = bool_8_column_to_array(col) elif dtype == DtypeKind.CATEGORICAL: columns[name], buf = categorical_column_to_dictionary(col) - elif dtype == DtypeKind.STRING: - columns[name], buf = string_column_to_ndarray(col) elif dtype == DtypeKind.DATETIME: - columns[name], buf = datetime_column_to_ndarray(col) + columns[name], buf = datetime_column_to_array(col) else: raise NotImplementedError(f"Data type {dtype} not handled yet") @@ -125,57 +146,115 @@ def protocol_df_chunk_to_pyarrow(df: DataFrameObject) -> pa.Table: pass -def primitive_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: +def column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: """ Convert a column holding one of the primitive dtypes to a PyArrow array. - A primitive type is one of: int, uint, float, bool. + A primitive type is one of: int, uint, float, bool (1 bit). + Parameters ---------- col : ColumnObject + Returns ------- tuple Tuple of pa.Array holding the data and the memory owner object that keeps the memory alive. """ - pass + buffers = col.get_buffers() + data = buffers_to_array(buffers, col.size(), col.describe_null, col.offset) + return data, buffers -def categorical_column_to_dictionary( - col: ColumnObject -) -> tuple[pa.Array, Any]: +def bool_8_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: """ - Convert a column holding categorical data to a pandas Series. + Convert a column holding boolean dtype to with bit width equal 8 + to a PyArrow array. + Parameters ---------- col : ColumnObject + Returns ------- tuple Tuple of pa.Array holding the data and the memory owner object that keeps the memory alive. """ - pass + buffers = col.get_buffers() + + # Data buffer + data_buff, data_dtype = buffers["data"] + offset = col.offset + data_bit_width = data_dtype[1] + + data = get_ndarray_from_buffer(data_buff, + data_bit_width, + offset) + + # Validity buffer + try: + validity_buff, validity_dtype = buffers["validity"] + except TypeError: + validity_buff = None + + null_kind, sentinel_val = col.describe_null + bytemask = None + + if validity_buff: + if null_kind in (ColumnNullType.USE_BYTEMASK, + ColumnNullType.USE_BITMASK): + + validity_bit_width = validity_dtype[1] + bytemask = get_ndarray_from_buffer(validity_buff, + validity_bit_width, + offset) + if sentinel_val == 0: + bytemask = np.invert(bytemask) + else: + raise NotImplementedError(f"{null_kind} null representation " + "is not yet supported for boolean " + "dtype.") + # Output + return pa.array(data, mask=bytemask), buffers -def string_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: +def categorical_column_to_dictionary( + col: ColumnObject +) -> tuple[pa.DictionaryArray, Any]: """ - Convert a column holding string data to a NumPy array. + Convert a column holding categorical data to a pa.DictionaryArray. + Parameters ---------- col : ColumnObject + Returns ------- tuple - Tuple of pa.Array holding the data and the memory owner object - that keeps the memory alive. + Tuple of pa.DictionaryArray holding the data and the memory owner + object that keeps the memory alive. """ - pass + categorical = col.describe_categorical + null_kind, sentinel_val = col.describe_null + if not categorical["is_dictionary"]: + raise NotImplementedError( + "Non-dictionary categoricals not supported yet") -def parse_datetime_format_str(format_str, data): - """Parse datetime `format_str` to interpret the `data`.""" - pass + cat_column = categorical["categories"] + dictionary = column_to_array(cat_column)[0] + + buffers = col.get_buffers() + indices = buffers_to_array( + buffers, col.size(), col.describe_null, col.offset) + + if null_kind == ColumnNullType.USE_SENTINEL: + bytemask = [value == sentinel_val for value in indices.to_pylist()] + indices = pa.array(indices.to_pylist(), mask=bytemask) + + dict_array = pa.DictionaryArray.from_arrays(indices, dictionary) + return dict_array, buffers def datetime_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: @@ -184,34 +263,118 @@ def datetime_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: Parameters ---------- col : ColumnObject + Returns ------- tuple Tuple of pa.Array holding the data and the memory owner object that keeps the memory alive. """ - pass + buffers = col.get_buffers() + data_buff, data_type = buffers["data"] + try: + validity_buff, validity_dtype = buffers["validity"] + except TypeError: + validity_buff = None + + format_str = data_type[2] + unit, tz = parse_datetime_format_str(format_str) + data_dtype = pa.timestamp(unit, tz=tz) + + # Data buffer + data_pa_buffer = pa.foreign_buffer(data_buff.ptr, data_buff.bufsize) + + # Validity buffer + validity_pa_buff = validity_buff + bytemask = None + if validity_pa_buff: + validity_pa_buff, bytemask = validity_buffer(validity_buff, + validity_dtype, + col.describe_null, + col.offset) + + # Constructing a pa.Array from data and validity buffer + array = pa.Array.from_buffers( + data_dtype, + col.size(), + [validity_pa_buff, data_pa_buffer], + offset=col.offset, + ) + + null_kind, sentinel_val = col.describe_null + if null_kind == ColumnNullType.USE_SENTINEL: + # pandas iNaT + if sentinel_val == -9223372036854775808: + bytemask = np.isnan(np.array([str(e) for e in array.to_pylist()], + dtype="datetime64[ns]")) + else: + raise NotImplementedError( + f"Sentinel value {sentinel_val} for datetime is not yet supported.") + # In case a bytemask was constructed with validity_buffer() call + # or with sentinel_value then we have to add the mask to the array + if bytemask is not None: + return pa.array(array.to_pylist(), mask=bytemask) + else: + return array -def buffer_to_array( - buffer: BufferObject, - dtype: tuple[DtypeKind, int, str, str], + +def parse_datetime_format_str(format_str): + """Parse datetime `format_str` to interpret the `data`.""" + # timestamp 'ts{unit}:tz' + timestamp_meta = re.match(r"ts([smun]):(.*)", format_str) + if timestamp_meta: + unit, tz = timestamp_meta.group(1), timestamp_meta.group(2) + if tz != "": + raise NotImplementedError("Timezones are not supported yet") + if unit != "s": + # the format string describes only a first letter of the unit, so + # add one extra letter to convert the unit to numpy-style: + # 'm' -> 'ms', 'u' -> 'us', 'n' -> 'ns' + unit += "s" + + return unit, tz + + # date 'td{Days/Ms}' + # date_meta = re.match(r"td([Dm])", format_str) + # if date_meta: + # unit = date_meta.group(1) + # if unit == "D": + # # NumPy doesn't support DAY unit, so converting days to seconds + # # (converting to uint64 to avoid overflow) + # data = data.astype(np.uint64) * (24 * 60 * 60) + # data = data.astype("datetime64[s]") + # elif unit == "m": + # data = data.astype("datetime64[ms]") + # else: + # raise NotImplementedError(f"Date unit is not supported: {unit}") + # return data + + raise NotImplementedError(f"DateTime kind is not supported: {format_str}") + + +def buffers_to_array( + buffers: ColumnBuffers, + length: int, + describe_null: ColumnNullType, offset: int = 0, - length: int | None = None, ) -> pa.Array: """ - Build a NumPy array from the passed buffer. + Build a PyArrow array from the passed buffer. + Parameters ---------- - buffer : BufferObject - Buffer to build a PyArrow array from. - dtype : tuple - Data type of the buffer conforming protocol dtypes format. + buffer : ColumnBuffers + Dictionary containing tuples of underlying buffers and + their associated dtype. + length : int + The number of values in the array. + describe_null: ColumnNullType + Null representation the column dtype uses, + as a tuple ``(kind, value)`` offset : int, default: 0 Number of elements to offset from the start of the buffer. - length : int, optional - If the buffer is a bit-mask, specifies a number of bits to read - from the buffer. Has no effect otherwise. + Returns ------- pa.Array @@ -220,26 +383,151 @@ def buffer_to_array( ----- The returned array doesn't own the memory. The caller of this function is responsible for keeping the memory owner object alive as long as - the returned NumPy array is being used. + the returned PyArrow array is being used. """ - pass + data_buff, data_type = buffers["data"] + try: + validity_buff, validity_dtype = buffers["validity"] + except TypeError: + validity_buff = None + try: + offset_buff, _ = buffers["offsets"] + except TypeError: + offset_buff = None + + kind, bit_width, _, _ = data_type + + data_dtype = _PYARROW_DTYPES.get(kind, {}).get(bit_width, None) + if data_dtype is None: + raise NotImplementedError( + f"Conversion for {data_type} is not yet supported.") + + # Construct a pyarrow Buffer + data_pa_buffer = pa.foreign_buffer(data_buff.ptr, data_buff.bufsize) + + # Construct a validity pyarrow Buffer or a bytemask, if exists + validity_pa_buff = validity_buff + bytemask = None + if validity_pa_buff: + validity_pa_buff, bytemask = validity_buffer(validity_buff, + validity_dtype, + describe_null, + offset) + + # Construct a pyarrow Array from buffers + if offset_buff: + # If an offset buffer exists, construct an offset pyarrow Buffer + # and add it to the construction of an array + offset_pa_buffer = pa.py_buffer(offset_buff._x) + array = pa.Array.from_buffers( + pa.large_string(), + length, + [validity_pa_buff, offset_pa_buffer, data_pa_buffer], + offset=offset, + ) + else: + array = pa.Array.from_buffers( + data_dtype, + length, + [validity_pa_buff, data_pa_buffer], + offset=offset, + ) + + # In case a bytemask was constructed with validity_buffer() call + # then we have to add the mask to the array + if bytemask is not None: + return pa.array(array.to_pylist(), mask=bytemask) + else: + return array + + +def validity_buffer( + validity_buff: BufferObject, + validity_dtype: Dtype, + describe_null: ColumnNullType, + offset: int = 0, +) -> tuple[pa.Buffer, np.ndarray]: + """ + Build a PyArrow buffer or a NumPy array from the passed buffer. + Parameters + ---------- + validity_buff : BufferObject + Tuple of underlying validity buffer and associated dtype. + validity_dtype : Dtype + Dtype description as a tuple ``(kind, bit-width, format string, + endianness)``. + describe_null : ColumnNullType + Null representation the column dtype uses, + as a tuple ``(kind, value)`` + offset : int, default: 0 + Number of elements to offset from the start of the buffer. -def bitmask_to_bool_array( - bitmask: np.ndarray, mask_length: int, first_byte_offset: int = 0 -) -> pa.Array: + Returns + ------- + tuple + Tuple of a pyarrow buffer and a np.ndarray defining a mask. + Ony one of both can be not None. + """ + null_kind, sentinel_val = describe_null + validity_kind, validity_bit_width, _, _ = validity_dtype + assert validity_kind == DtypeKind.BOOL + + bytemask = None + if null_kind == ColumnNullType.USE_BYTEMASK: + + bytemask = get_ndarray_from_buffer(validity_buff, + validity_bit_width, + offset) + if sentinel_val == 0: + bytemask = np.invert(bytemask) + validity_pa_buff = None + + elif null_kind == ColumnNullType.USE_BITMASK and sentinel_val == 0: + validity_pa_buff = pa.foreign_buffer( + validity_buff.ptr, validity_buff.bufsize) + else: + raise NotImplementedError( + f"{describe_null} null representation is not yet supported.") + + return validity_pa_buff, bytemask + + +def get_ndarray_from_buffer( + validity_buff: BufferObject, + bit_width: int, + offset: int = 0, +) -> np.ndarray: """ - Convert bit-mask to a boolean NumPy array. + Build a NumPy array from the passed buffer. + Parameters ---------- - bitmask : np.ndarray[uint8] - NumPy array of uint8 dtype representing the bitmask. - mask_length : int - Number of elements in the mask to interpret. - first_byte_offset : int, default: 0 - Number of elements to offset from the start of the first byte. + validity_buff : BufferObject + Tuple of underlying validity buffer and associated dtype. + bit_width : int + The number of bits of the buffer dtype as an integer. + sentinel_value : int, default: 0 + The value (0 or 1) indicating a missing value in the bit/bytemask. + offset : int, default: 0 + Number of elements to offset from the start of the buffer. + Returns ------- - pa.Array[bool] + np.ndarray """ - pass \ No newline at end of file + # TODO: No DLPack yet, so need to construct a new ndarray from + # the data pointer and size in the buffer plus the dtype on the + # column. Use DLPack as NumPy supports it since + # https://github.com/numpy/numpy/pull/19083 + ctypes_type = np.ctypeslib.as_ctypes_type(bool) + data_pointer = ctypes.cast( + validity_buff.ptr + (offset * bit_width // 8), + ctypes.POINTER(ctypes_type) + ) + + data = np.ctypeslib.as_array( + data_pointer, shape=(validity_buff.bufsize // (bit_width // 8),) + ) + + return data From c356cd1460629ede7c8636adbae745a402ee74aa Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Thu, 1 Dec 2022 16:36:30 +0100 Subject: [PATCH 03/49] Fix linter errors --- python/pyarrow/interchange/from_dataframe.py | 23 ++++++++++---------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index dace516d79e..22b74de11ef 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -204,17 +204,17 @@ def bool_8_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: if validity_buff: if null_kind in (ColumnNullType.USE_BYTEMASK, ColumnNullType.USE_BITMASK): - + validity_bit_width = validity_dtype[1] - bytemask = get_ndarray_from_buffer(validity_buff, - validity_bit_width, - offset) + bytemask = get_ndarray_from_buffer(validity_buff, + validity_bit_width, + offset) if sentinel_val == 0: bytemask = np.invert(bytemask) else: raise NotImplementedError(f"{null_kind} null representation " - "is not yet supported for boolean " - "dtype.") + "is not yet supported for boolean " + "dtype.") # Output return pa.array(data, mask=bytemask), buffers @@ -292,7 +292,7 @@ def datetime_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: validity_dtype, col.describe_null, col.offset) - + # Constructing a pa.Array from data and validity buffer array = pa.Array.from_buffers( data_dtype, @@ -309,7 +309,8 @@ def datetime_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: dtype="datetime64[ns]")) else: raise NotImplementedError( - f"Sentinel value {sentinel_val} for datetime is not yet supported.") + f"Sentinel value {sentinel_val} for datetime is not yet " + "supported.") # In case a bytemask was constructed with validity_buffer() call # or with sentinel_value then we have to add the mask to the array @@ -401,7 +402,7 @@ def buffers_to_array( if data_dtype is None: raise NotImplementedError( f"Conversion for {data_type} is not yet supported.") - + # Construct a pyarrow Buffer data_pa_buffer = pa.foreign_buffer(data_buff.ptr, data_buff.bufsize) @@ -475,12 +476,12 @@ def validity_buffer( bytemask = None if null_kind == ColumnNullType.USE_BYTEMASK: - + bytemask = get_ndarray_from_buffer(validity_buff, validity_bit_width, offset) if sentinel_val == 0: - bytemask = np.invert(bytemask) + bytemask = np.invert(bytemask) validity_pa_buff = None elif null_kind == ColumnNullType.USE_BITMASK and sentinel_val == 0: From 8fb50c5e9e6d71adfaa9a575d0271183dc6fd0d2 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Mon, 5 Dec 2022 14:55:08 +0100 Subject: [PATCH 04/49] Add from_dataframe method details --- python/pyarrow/interchange/from_dataframe.py | 25 +++++++++++++------- 1 file changed, 17 insertions(+), 8 deletions(-) diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index 22b74de11ef..3d9a392a0b5 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -101,21 +101,30 @@ def _from_dataframe(df: DataFrameObject, allow_copy=True): ------- pa.Table """ - pass + batches = [] + for chunk in df.get_chunks(): + batch = protocol_df_chunk_to_pyarrow(chunk) + batches.append(batch) + + table = pa.Table.from_batches(batches) + + return table def protocol_df_chunk_to_pyarrow(df: DataFrameObject) -> pa.Table: """ - Convert interchange protocol chunk to ``pd.DataFrame``. + Convert interchange protocol chunk to ``pa.RecordBatch``. + Parameters ---------- df : DataFrameObject + Returns ------- - pa.Table + pa.RecordBatch """ - # We need a dict of columns here, with each column being a NumPy array - # (at least for now, deal with non-NumPy dtypes later). + # We need a dict of columns here, with each column being a PyArrow + # or NumPy array. columns: dict[str, Any] = {} buffers = [] # hold on to buffers, keeps memory alive for name in df.column_names(): @@ -143,7 +152,7 @@ def protocol_df_chunk_to_pyarrow(df: DataFrameObject) -> pa.Table: buffers.append(buf) - pass + return pa.RecordBatch.from_pydict(columns) def column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: @@ -315,9 +324,9 @@ def datetime_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: # In case a bytemask was constructed with validity_buffer() call # or with sentinel_value then we have to add the mask to the array if bytemask is not None: - return pa.array(array.to_pylist(), mask=bytemask) + return pa.array(array.to_pylist(), mask=bytemask), buffers else: - return array + return array, buffers def parse_datetime_format_str(format_str): From 4fab43bfae567eb7bcab63eba1aa82e949006bea Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Tue, 6 Dec 2022 10:12:36 +0100 Subject: [PATCH 05/49] Add tests for from_dataframe / pandas roundtrip --- .../pyarrow/tests/interchange/test_extra.py | 79 ++++++++++++++++++- .../interchange/test_interchange_spec.py | 5 +- 2 files changed, 80 insertions(+), 4 deletions(-) diff --git a/python/pyarrow/tests/interchange/test_extra.py b/python/pyarrow/tests/interchange/test_extra.py index 4181b117be6..68b4ac05ff1 100644 --- a/python/pyarrow/tests/interchange/test_extra.py +++ b/python/pyarrow/tests/interchange/test_extra.py @@ -25,11 +25,11 @@ ColumnNullType, DtypeKind, ) +from pyarrow.interchange.from_dataframe import from_dataframe try: import pandas as pd import pandas.testing as tm - from pandas.core.interchange.from_dataframe import from_dataframe except ImportError: pass @@ -74,14 +74,87 @@ def test_offset_of_sliced_array(): if Version(pd.__version__) < Version("1.5.0"): pytest.skip("__dataframe__ added to pandas in 1.5.0") + from pandas.core.interchange.from_dataframe import ( + from_dataframe as pandas_from_dataframe + ) + arr = pa.array([1, 2, 3, 4]) arr_sliced = arr.slice(2, 2) table = pa.table([arr], names=["arr"]) table_sliced = pa.table([arr_sliced], names=["arr_sliced"]) - df = from_dataframe(table) - df_sliced = from_dataframe(table_sliced) + df = pandas_from_dataframe(table) + df_sliced = pandas_from_dataframe(table_sliced) tm.assert_series_equal(df["arr"][2:4], df_sliced["arr_sliced"], check_index=False, check_names=False) + + +@pytest.mark.pandas +def test_categorical_roundtrip(): + arr = ["Mon", "Tue", "Mon", "Wed", "Mon", "Thu", "Fri", "Sat", "Sun"] + table = pa.table( + {"weekday": pa.array(arr).dictionary_encode()} + ) + + pandas_df = table.to_pandas() + result = from_dataframe(pandas_df) + + # Checking equality for the values + # As the dtype of the indices is changed from int32 in pa.Table + # to int64 in pandas interchange protocol implementation + assert result[0].chunk(0).dictionary == table[0].chunk(0).dictionary + + table_protocol = table.__dataframe__() + result_protocol = result.__dataframe__() + + assert table_protocol.num_columns() == result_protocol.num_columns() + assert table_protocol.num_rows() == result_protocol.num_rows() + assert table_protocol.num_chunks() == result_protocol.num_chunks() + assert table_protocol.column_names() == result_protocol.column_names() + + col_table = table_protocol.get_column(0) + col_result = result_protocol.get_column(0) + + assert col_result.dtype[0] == DtypeKind.CATEGORICAL + assert col_result.dtype[0] == col_table.dtype[0] + assert col_result.size == col_table.size + assert col_result.offset == col_table.offset + + desc_cat_table = col_result.describe_categorical + desc_cat_result = col_result.describe_categorical + + assert desc_cat_table["is_ordered"] == desc_cat_result["is_ordered"] + assert desc_cat_table["is_dictionary"] == desc_cat_result["is_dictionary"] + assert isinstance(desc_cat_result["categories"]._col, pa.Array) + + +@pytest.mark.pandas +def test_pandas_roundtrip(): + from datetime import datetime as dt + + table = pa.table( + { + "a": [1, 2, 3, 4], # dtype kind INT = 0 + "b": [3, 4, 5, 6], # dtype kind INT = 0 + "c": [1.5, 2.5, 3.5, 4.5], # dtype kind FLOAT = 2 + "d": [9, 10, 11, 12], # dtype kind INT = 0 + "e": [True, True, False, False], # dtype kind BOOLEAN = 20 + "f": ["a", "", "c", "d"], # dtype kind STRING = 21 + "g": [dt(2007, 7, 13), dt(2007, 7, 14), + dt(2007, 7, 15), dt(2007, 7, 16)] # dtype kind DATETIME = 22 + } + ) + + pandas_df = table.to_pandas() + result = from_dataframe(pandas_df) + assert table.equals(result) + + table_protocol = table.__dataframe__() + result_protocol = result.__dataframe__() + + assert table_protocol.num_columns() == result_protocol.num_columns() + assert table_protocol.num_rows() == result_protocol.num_rows() + assert table_protocol.num_chunks() == result_protocol.num_chunks() + assert table_protocol.column_names() == result_protocol.column_names() diff --git a/python/pyarrow/tests/interchange/test_interchange_spec.py b/python/pyarrow/tests/interchange/test_interchange_spec.py index 425e8f7f95d..09fcb9c0f09 100644 --- a/python/pyarrow/tests/interchange/test_interchange_spec.py +++ b/python/pyarrow/tests/interchange/test_interchange_spec.py @@ -44,6 +44,7 @@ def test_only_one_dtype(test_data): def test_mixed_dtypes(): + from datetime import datetime as dt table = pa.table( { "a": [1, 2, 3], # dtype kind INT = 0 @@ -52,13 +53,15 @@ def test_mixed_dtypes(): "d": [9, 10, 11], # dtype kind INT = 0 "e": [True, False, True], # dtype kind BOOLEAN = 20 "f": ["a", "", "c"], # dtype kind STRING = 21 + "g": [dt(2007, 7, 13), dt(2007, 7, 14), + dt(2007, 7, 15)] # dtype kind DATETIME = 22 } ) df = table.__dataframe__() # for meanings of dtype[0] see the spec; we cannot import the # spec here as this file is expected to be vendored *anywhere*; # values for dtype[0] are explained above - columns = {"a": 0, "b": 0, "c": 2, "d": 0, "e": 20, "f": 21} + columns = {"a": 0, "b": 0, "c": 2, "d": 0, "e": 20, "f": 21, "g": 22} for column, kind in columns.items(): col = df.get_column_by_name(column) From 47ce2d63046806c12f55ad540171887ebf353926 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Tue, 6 Dec 2022 12:23:22 +0100 Subject: [PATCH 06/49] Skip from_dataframe tests for older pandas versions --- python/pyarrow/tests/interchange/test_extra.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python/pyarrow/tests/interchange/test_extra.py b/python/pyarrow/tests/interchange/test_extra.py index 68b4ac05ff1..b85091923e8 100644 --- a/python/pyarrow/tests/interchange/test_extra.py +++ b/python/pyarrow/tests/interchange/test_extra.py @@ -93,6 +93,8 @@ def test_offset_of_sliced_array(): @pytest.mark.pandas def test_categorical_roundtrip(): + if Version(pd.__version__) < Version("1.5.0"): + pytest.skip("__dataframe__ added to pandas in 1.5.0") arr = ["Mon", "Tue", "Mon", "Wed", "Mon", "Thu", "Fri", "Sat", "Sun"] table = pa.table( {"weekday": pa.array(arr).dictionary_encode()} @@ -132,6 +134,8 @@ def test_categorical_roundtrip(): @pytest.mark.pandas def test_pandas_roundtrip(): + if Version(pd.__version__) < Version("1.5.0"): + pytest.skip("__dataframe__ added to pandas in 1.5.0") from datetime import datetime as dt table = pa.table( From 1c2955f281d68a9639106582167480f877d7a8e5 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Mon, 12 Dec 2022 17:35:13 +0100 Subject: [PATCH 07/49] Add support for LargeStringArrays in Column class --- python/pyarrow/interchange/column.py | 13 ++++++++++-- .../pyarrow/tests/interchange/test_extra.py | 21 +++++++++++++++++++ 2 files changed, 32 insertions(+), 2 deletions(-) diff --git a/python/pyarrow/interchange/column.py b/python/pyarrow/interchange/column.py index a5c9d4a2833..416314095f9 100644 --- a/python/pyarrow/interchange/column.py +++ b/python/pyarrow/interchange/column.py @@ -260,7 +260,12 @@ def dtype(self) -> Tuple[DtypeKind, int, str, str]: except ValueError: # in case of a variable-length strings bit_width = 8 - if pa.types.is_timestamp(dtype): + if pa.types.is_large_string(dtype): + # format string needs to be changed from "U" to "u" + # in case of large strings (make an issue in pandas?) + kind = DtypeKind.STRING + return kind, bit_width, "u", Endianness.NATIVE + elif pa.types.is_timestamp(dtype): kind = DtypeKind.DATETIME ts = dtype.unit[0] tz = dtype.tz if dtype.tz else "" @@ -510,5 +515,9 @@ def _get_offsets_buffer(self) -> Tuple[_PyArrowBuffer, Any]: ) elif n == 3: # Define the dtype of the returned buffer - dtype = (DtypeKind.INT, 32, "i", Endianness.NATIVE) + dtype = self._col.type + if pa.types.is_large_string(dtype): + dtype = (DtypeKind.INT, 64, "i", Endianness.NATIVE) + else: + dtype = (DtypeKind.INT, 32, "i", Endianness.NATIVE) return _PyArrowBuffer(array.buffers()[1]), dtype diff --git a/python/pyarrow/tests/interchange/test_extra.py b/python/pyarrow/tests/interchange/test_extra.py index b85091923e8..5a99e11b943 100644 --- a/python/pyarrow/tests/interchange/test_extra.py +++ b/python/pyarrow/tests/interchange/test_extra.py @@ -16,6 +16,7 @@ # under the License. from datetime import datetime as dt +import numpy as np import pyarrow as pa from pyarrow.vendored.version import Version import pytest @@ -162,3 +163,23 @@ def test_pandas_roundtrip(): assert table_protocol.num_rows() == result_protocol.num_rows() assert table_protocol.num_chunks() == result_protocol.num_chunks() assert table_protocol.column_names() == result_protocol.column_names() + + +@pytest.mark.pandas +def test_large_string(): + if Version(pd.__version__) < Version("1.5.0"): + pytest.skip("__dataframe__ added to pandas in 1.5.0") + + from pandas.core.interchange.from_dataframe import ( + from_dataframe as pandas_from_dataframe + ) + + data = np.array([b'x'*1024]*(3*1024**2), dtype='object') # 3GB bytes data + arr = pa.array(data, type=pa.large_string()) + table = pa.table([arr], names = ["large_string"]) + + exchange_df = table.__dataframe__() + result = pandas_from_dataframe(exchange_df) + + assert result["string"][0] == data[0].decode() + assert result.size == 3*1024**2 From 0517a6d0de05f7fabe3ee5f810e7e4e9747524d2 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Tue, 13 Dec 2022 05:17:27 +0100 Subject: [PATCH 08/49] Add test for uint and make changes to test_offset_of_sliced_array() and test_large_string() --- .../pyarrow/tests/interchange/test_extra.py | 46 +++++++++++++++++-- 1 file changed, 41 insertions(+), 5 deletions(-) diff --git a/python/pyarrow/tests/interchange/test_extra.py b/python/pyarrow/tests/interchange/test_extra.py index 5a99e11b943..fa2c9afb257 100644 --- a/python/pyarrow/tests/interchange/test_extra.py +++ b/python/pyarrow/tests/interchange/test_extra.py @@ -85,6 +85,12 @@ def test_offset_of_sliced_array(): table = pa.table([arr], names=["arr"]) table_sliced = pa.table([arr_sliced], names=["arr_sliced"]) + # Creating a table from sliced array makes a copy? + # And so the offset of the sliced column in a table is 0 + # df = table_sliced.__dataframe__() + # col = df.get_column(0) + # assert col.offset == 2 + df = pandas_from_dataframe(table) df_sliced = pandas_from_dataframe(table_sliced) @@ -176,10 +182,40 @@ def test_large_string(): data = np.array([b'x'*1024]*(3*1024**2), dtype='object') # 3GB bytes data arr = pa.array(data, type=pa.large_string()) - table = pa.table([arr], names = ["large_string"]) - - exchange_df = table.__dataframe__() - result = pandas_from_dataframe(exchange_df) + table = pa.table([arr], names=["large_string"]) + result = pandas_from_dataframe(table) - assert result["string"][0] == data[0].decode() + assert result["large_string"][0] == data[0].decode() assert result.size == 3*1024**2 + + +def test_uint(): + arr = pa.array([1, 2, 3, None], type=pa.uint16()) + table = pa.table([arr], names=["uint"]) + df = table.__dataframe__() + col = df.get_column(0) + + assert col.null_count == 1 + assert isinstance(col.null_count, int) + assert col.size == 4 + assert col.offset == 0 + + assert col.dtype[0] == DtypeKind.UINT + assert col.dtype[1] == 16 + + +@pytest.mark.pandas +def test_uint_roundtrip(): + if Version(pd.__version__) < Version("1.5.0"): + pytest.skip("__dataframe__ added to pandas in 1.5.0") + + from pandas.core.interchange.from_dataframe import ( + from_dataframe as pandas_from_dataframe + ) + + arr = pa.array([1, 2, 3, 4], type=pa.uint16()) + table = pa.table([arr], names=["uint"]) + result = pandas_from_dataframe(table) + result_to_table = pa.Table.from_pandas(result) + + assert table[0] == result_to_table[0] From a7313fba77163f08dc1304f3b1a91e9c5b08d793 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Tue, 13 Dec 2022 05:23:37 +0100 Subject: [PATCH 09/49] Prefix table metadata with pyarrow. --- python/pyarrow/interchange/dataframe.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyarrow/interchange/dataframe.py b/python/pyarrow/interchange/dataframe.py index 278ee102ec9..fdaf3849546 100644 --- a/python/pyarrow/interchange/dataframe.py +++ b/python/pyarrow/interchange/dataframe.py @@ -87,7 +87,7 @@ def metadata(self) -> dict[str, Any]: # The metadata for the data frame, as a dictionary with string keys. # Add schema metadata here (pandas metadata or custom metadata) if self._df.schema.metadata: - schema_metadata = {k.decode('utf8'): v.decode('utf8') + schema_metadata = {"pyarrow." + k.decode('utf8'): v.decode('utf8') for k, v in self._df.schema.metadata.items()} return schema_metadata else: From 9583672a235dcf916c39ca122971dc5037148048 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Tue, 13 Dec 2022 05:37:20 +0100 Subject: [PATCH 10/49] Update from_dataframe method --- python/pyarrow/interchange/from_dataframe.py | 46 +++++++------------- 1 file changed, 16 insertions(+), 30 deletions(-) diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index 3d9a392a0b5..c5e1e4d9178 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -55,8 +55,7 @@ 32: pa.float32(), 64: pa.float64()}, DtypeKind.BOOL: {1: pa.bool_()}, - DtypeKind.STRING: {1: pa.string(), - 8: pa.large_string()}, + DtypeKind.STRING: {8: pa.string()}, } @@ -107,7 +106,6 @@ def _from_dataframe(df: DataFrameObject, allow_copy=True): batches.append(batch) table = pa.Table.from_batches(batches) - return table @@ -177,8 +175,8 @@ def column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: def bool_8_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: """ - Convert a column holding boolean dtype to with bit width equal 8 - to a PyArrow array. + Convert a column holding boolean dtype with bit width = 8 to a + PyArrow array. Parameters ---------- @@ -197,9 +195,9 @@ def bool_8_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: offset = col.offset data_bit_width = data_dtype[1] - data = get_ndarray_from_buffer(data_buff, - data_bit_width, - offset) + data = buffer_to_ndarray(data_buff, + data_bit_width, + offset) # Validity buffer try: @@ -215,9 +213,9 @@ def bool_8_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: ColumnNullType.USE_BITMASK): validity_bit_width = validity_dtype[1] - bytemask = get_ndarray_from_buffer(validity_buff, - validity_bit_width, - offset) + bytemask = buffer_to_ndarray(validity_buff, + validity_bit_width, + offset) if sentinel_val == 0: bytemask = np.invert(bytemask) else: @@ -269,6 +267,7 @@ def categorical_column_to_dictionary( def datetime_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: """ Convert a column holding DateTime data to a NumPy array. + Parameters ---------- col : ColumnObject @@ -331,6 +330,7 @@ def datetime_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: def parse_datetime_format_str(format_str): """Parse datetime `format_str` to interpret the `data`.""" + # timestamp 'ts{unit}:tz' timestamp_meta = re.match(r"ts([smun]):(.*)", format_str) if timestamp_meta: @@ -345,20 +345,8 @@ def parse_datetime_format_str(format_str): return unit, tz + # TODO # date 'td{Days/Ms}' - # date_meta = re.match(r"td([Dm])", format_str) - # if date_meta: - # unit = date_meta.group(1) - # if unit == "D": - # # NumPy doesn't support DAY unit, so converting days to seconds - # # (converting to uint64 to avoid overflow) - # data = data.astype(np.uint64) * (24 * 60 * 60) - # data = data.astype("datetime64[s]") - # elif unit == "m": - # data = data.astype("datetime64[ms]") - # else: - # raise NotImplementedError(f"Date unit is not supported: {unit}") - # return data raise NotImplementedError(f"DateTime kind is not supported: {format_str}") @@ -486,9 +474,9 @@ def validity_buffer( bytemask = None if null_kind == ColumnNullType.USE_BYTEMASK: - bytemask = get_ndarray_from_buffer(validity_buff, - validity_bit_width, - offset) + bytemask = buffer_to_ndarray(validity_buff, + validity_bit_width, + offset) if sentinel_val == 0: bytemask = np.invert(bytemask) validity_pa_buff = None @@ -503,7 +491,7 @@ def validity_buffer( return validity_pa_buff, bytemask -def get_ndarray_from_buffer( +def buffer_to_ndarray( validity_buff: BufferObject, bit_width: int, offset: int = 0, @@ -517,8 +505,6 @@ def get_ndarray_from_buffer( Tuple of underlying validity buffer and associated dtype. bit_width : int The number of bits of the buffer dtype as an integer. - sentinel_value : int, default: 0 - The value (0 or 1) indicating a missing value in the bit/bytemask. offset : int, default: 0 Number of elements to offset from the start of the buffer. From 9e8733c483e8b44741fd86cbf4c8f950b8c91020 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Tue, 13 Dec 2022 16:17:59 +0100 Subject: [PATCH 11/49] Try to add warnings to places where copies of data are being made --- python/pyarrow/interchange/column.py | 50 ++++++---------- python/pyarrow/interchange/from_dataframe.py | 60 +++++++++++++++---- .../pyarrow/tests/interchange/test_extra.py | 30 +++++----- 3 files changed, 79 insertions(+), 61 deletions(-) diff --git a/python/pyarrow/interchange/column.py b/python/pyarrow/interchange/column.py index 416314095f9..e4b0247b304 100644 --- a/python/pyarrow/interchange/column.py +++ b/python/pyarrow/interchange/column.py @@ -197,6 +197,16 @@ def __init__( Handles PyArrow Arrays and ChunkedArrays. """ # Store the column as a private attribute + if isinstance(column, pa.ChunkedArray): + if column.num_chunks == 1: + column = column.chunk(0) + else: + if not allow_copy: + raise RuntimeError( + "Chunks will be combined and a copy is required which " + "is forbidden by allow_copy=False" + ) + column = column.combine_chunks() self._col = column self._allow_copy = allow_copy @@ -219,12 +229,7 @@ def offset(self) -> int: equal size M (only the last chunk may be shorter), ``offset = n * M``, ``n = 0 .. N-1``. """ - if isinstance(self._col, pa.Array): - return self._col.offset - else: - # ChunkedArray gets copied with `combine_chunks` so the offset will - # always be 0 - return 0 + return self._col.offset @property def dtype(self) -> Tuple[DtypeKind, int, str, str]: @@ -316,11 +321,7 @@ def describe_categorical(self) -> CategoricalDescription: categorical. TBD: are there any other in-memory representations that are needed? """ - if isinstance(self._col, pa.ChunkedArray): - arr = self._col.combine_chunks() - else: - arr = self._col - + arr = self._col if not pa.types.is_dictionary(arr.type): raise TypeError( "describe_categorical only works on a column with " @@ -369,11 +370,7 @@ def num_chunks(self) -> int: """ Return the number of chunks the column consists of. """ - if isinstance(self._col, pa.Array): - n_chunks = 1 - else: - n_chunks = self._col.num_chunks - return n_chunks + return 1 def get_chunks( self, n_chunks: Optional[int] = None @@ -387,11 +384,7 @@ def get_chunks( if self.size % n_chunks != 0: chunk_size += 1 - if isinstance(self._col, pa.ChunkedArray): - array = self._col.combine_chunks() - else: - array = self._col - + array = self._col i = 0 for start in range(0, chunk_size * n_chunks, chunk_size): yield _PyArrowColumn( @@ -451,10 +444,7 @@ def _get_data_buffer( Return the buffer containing the data and the buffer's associated dtype. """ - if isinstance(self._col, pa.ChunkedArray): - array = self._col.combine_chunks() - else: - array = self._col + array = self._col dtype = self.dtype # In case of boolean arrays, cast to uint8 array @@ -484,10 +474,7 @@ def _get_validity_buffer(self) -> Tuple[_PyArrowBuffer, Any]: """ # Define the dtype of the returned buffer dtype = (DtypeKind.BOOL, 1, "b", Endianness.NATIVE) - if isinstance(self._col, pa.ChunkedArray): - array = self._col.combine_chunks() - else: - array = self._col + array = self._col buff = array.buffers()[0] if buff: return _PyArrowBuffer(buff), dtype @@ -503,10 +490,7 @@ def _get_offsets_buffer(self) -> Tuple[_PyArrowBuffer, Any]: Raises NoBufferPresent if the data buffer does not have an associated offsets buffer. """ - if isinstance(self._col, pa.ChunkedArray): - array = self._col.combine_chunks() - else: - array = self._col + array = self._col n = len(array.buffers()) if n == 2: raise NoBufferPresent( diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index c5e1e4d9178..05203ff08c3 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -102,14 +102,17 @@ def _from_dataframe(df: DataFrameObject, allow_copy=True): """ batches = [] for chunk in df.get_chunks(): - batch = protocol_df_chunk_to_pyarrow(chunk) + batch = protocol_df_chunk_to_pyarrow(chunk, allow_copy) batches.append(batch) table = pa.Table.from_batches(batches) return table -def protocol_df_chunk_to_pyarrow(df: DataFrameObject) -> pa.Table: +def protocol_df_chunk_to_pyarrow( + df: DataFrameObject, + allow_copy: bool = True +) -> pa.Table: """ Convert interchange protocol chunk to ``pa.RecordBatch``. @@ -138,13 +141,14 @@ def protocol_df_chunk_to_pyarrow(df: DataFrameObject) -> pa.Table: DtypeKind.FLOAT, DtypeKind.STRING, ): - columns[name], buf = column_to_array(col) + columns[name], buf = column_to_array(col, allow_copy) elif dtype == DtypeKind.BOOL: - columns[name], buf = bool_8_column_to_array(col) + columns[name], buf = bool_8_column_to_array(col, allow_copy) elif dtype == DtypeKind.CATEGORICAL: - columns[name], buf = categorical_column_to_dictionary(col) + columns[name], buf = categorical_column_to_dictionary( + col, allow_copy) elif dtype == DtypeKind.DATETIME: - columns[name], buf = datetime_column_to_array(col) + columns[name], buf = datetime_column_to_array(col, allow_copy) else: raise NotImplementedError(f"Data type {dtype} not handled yet") @@ -153,7 +157,10 @@ def protocol_df_chunk_to_pyarrow(df: DataFrameObject) -> pa.Table: return pa.RecordBatch.from_pydict(columns) -def column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: +def column_to_array( + col: ColumnObject, + allow_copy: bool = True +) -> tuple[pa.Array, Any]: """ Convert a column holding one of the primitive dtypes to a PyArrow array. A primitive type is one of: int, uint, float, bool (1 bit). @@ -169,11 +176,15 @@ def column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: that keeps the memory alive. """ buffers = col.get_buffers() - data = buffers_to_array(buffers, col.size(), col.describe_null, col.offset) + data = buffers_to_array(buffers, col.size(), + col.describe_null, col.offset, allow_copy) return data, buffers -def bool_8_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: +def bool_8_column_to_array( + col: ColumnObject, + allow_copy=True +) -> tuple[pa.Array, Any]: """ Convert a column holding boolean dtype with bit width = 8 to a PyArrow array. @@ -188,6 +199,11 @@ def bool_8_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: Tuple of pa.Array holding the data and the memory owner object that keeps the memory alive. """ + if not allow_copy: + raise RuntimeError( + "PyArrow Array will always be created out of a NumPy ndarray " + "and a copy is required which is forbidden by allow_copy=False" + ) buffers = col.get_buffers() # Data buffer @@ -227,7 +243,8 @@ def bool_8_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: def categorical_column_to_dictionary( - col: ColumnObject + col: ColumnObject, + allow_copy: bool = False ) -> tuple[pa.DictionaryArray, Any]: """ Convert a column holding categorical data to a pa.DictionaryArray. @@ -254,9 +271,14 @@ def categorical_column_to_dictionary( buffers = col.get_buffers() indices = buffers_to_array( - buffers, col.size(), col.describe_null, col.offset) + buffers, col.size(), col.describe_null, col.offset, allow_copy) if null_kind == ColumnNullType.USE_SENTINEL: + if not allow_copy: + raise RuntimeError( + "PyArrow Array will always be created out of a NumPy ndarray " + "and a copy is required which is forbidden by allow_copy=False" + ) bytemask = [value == sentinel_val for value in indices.to_pylist()] indices = pa.array(indices.to_pylist(), mask=bytemask) @@ -264,7 +286,10 @@ def categorical_column_to_dictionary( return dict_array, buffers -def datetime_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: +def datetime_column_to_array( + col: ColumnObject, + allow_copy: bool = False +) -> tuple[pa.Array, Any]: """ Convert a column holding DateTime data to a NumPy array. @@ -323,6 +348,11 @@ def datetime_column_to_array(col: ColumnObject) -> tuple[pa.Array, Any]: # In case a bytemask was constructed with validity_buffer() call # or with sentinel_value then we have to add the mask to the array if bytemask is not None: + if not allow_copy: + raise RuntimeError( + "PyArrow Array will always be created out of a NumPy ndarray " + "and a copy is required which is forbidden by allow_copy=False" + ) return pa.array(array.to_pylist(), mask=bytemask), buffers else: return array, buffers @@ -356,6 +386,7 @@ def buffers_to_array( length: int, describe_null: ColumnNullType, offset: int = 0, + allow_copy: bool = False ) -> pa.Array: """ Build a PyArrow array from the passed buffer. @@ -434,6 +465,11 @@ def buffers_to_array( # In case a bytemask was constructed with validity_buffer() call # then we have to add the mask to the array if bytemask is not None: + if not allow_copy: + raise RuntimeError( + "PyArrow Array will always be created out of a NumPy ndarray " + "and a copy is required which is forbidden by allow_copy=False" + ) return pa.array(array.to_pylist(), mask=bytemask) else: return array diff --git a/python/pyarrow/tests/interchange/test_extra.py b/python/pyarrow/tests/interchange/test_extra.py index fa2c9afb257..552e5cb563c 100644 --- a/python/pyarrow/tests/interchange/test_extra.py +++ b/python/pyarrow/tests/interchange/test_extra.py @@ -30,7 +30,7 @@ try: import pandas as pd - import pandas.testing as tm + # import pandas.testing as tm except ImportError: pass @@ -72,30 +72,28 @@ def test_array_to_pyarrowcolumn(test_data, kind): @pytest.mark.pandas def test_offset_of_sliced_array(): - if Version(pd.__version__) < Version("1.5.0"): - pytest.skip("__dataframe__ added to pandas in 1.5.0") + # if Version(pd.__version__) < Version("1.5.0"): + # pytest.skip("__dataframe__ added to pandas in 1.5.0") - from pandas.core.interchange.from_dataframe import ( - from_dataframe as pandas_from_dataframe - ) + # from pandas.core.interchange.from_dataframe import ( + # from_dataframe as pandas_from_dataframe + # ) arr = pa.array([1, 2, 3, 4]) arr_sliced = arr.slice(2, 2) - table = pa.table([arr], names=["arr"]) + # table = pa.table([arr], names=["arr"]) table_sliced = pa.table([arr_sliced], names=["arr_sliced"]) - # Creating a table from sliced array makes a copy? - # And so the offset of the sliced column in a table is 0 - # df = table_sliced.__dataframe__() - # col = df.get_column(0) - # assert col.offset == 2 + col = table_sliced.__dataframe__().get_column(0) + assert col.offset == 2 - df = pandas_from_dataframe(table) - df_sliced = pandas_from_dataframe(table_sliced) + # TODO: Investigate why this part is not working currently! + # df = pandas_from_dataframe(table) + # df_sliced = pandas_from_dataframe(table_sliced) - tm.assert_series_equal(df["arr"][2:4], df_sliced["arr_sliced"], - check_index=False, check_names=False) + # tm.assert_series_equal(df["arr"][2:4], df_sliced["arr_sliced"], + # check_index=False, check_names=False) @pytest.mark.pandas From 855ec8a1d55577d896477f8bce3fb57ea563d173 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Thu, 15 Dec 2022 15:55:56 +0100 Subject: [PATCH 12/49] Update python/pyarrow/interchange/column.py Co-authored-by: Joris Van den Bossche --- python/pyarrow/interchange/column.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyarrow/interchange/column.py b/python/pyarrow/interchange/column.py index e4b0247b304..a14eb2f91d7 100644 --- a/python/pyarrow/interchange/column.py +++ b/python/pyarrow/interchange/column.py @@ -501,7 +501,7 @@ def _get_offsets_buffer(self) -> Tuple[_PyArrowBuffer, Any]: # Define the dtype of the returned buffer dtype = self._col.type if pa.types.is_large_string(dtype): - dtype = (DtypeKind.INT, 64, "i", Endianness.NATIVE) + dtype = (DtypeKind.INT, 64, "l", Endianness.NATIVE) else: dtype = (DtypeKind.INT, 32, "i", Endianness.NATIVE) return _PyArrowBuffer(array.buffers()[1]), dtype From beec5aa0a80c37ad26aff54b9ae1426bd160d602 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Mon, 19 Dec 2022 08:45:13 +0100 Subject: [PATCH 13/49] Expose from_dataframe in interchange/__init__.py --- python/pyarrow/interchange/__init__.py | 2 ++ python/pyarrow/tests/interchange/test_extra.py | 6 +++--- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/python/pyarrow/interchange/__init__.py b/python/pyarrow/interchange/__init__.py index 13a83393a91..685556171ce 100644 --- a/python/pyarrow/interchange/__init__.py +++ b/python/pyarrow/interchange/__init__.py @@ -14,3 +14,5 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. + +from .from_dataframe import from_dataframe diff --git a/python/pyarrow/tests/interchange/test_extra.py b/python/pyarrow/tests/interchange/test_extra.py index 552e5cb563c..ec0f7e0015a 100644 --- a/python/pyarrow/tests/interchange/test_extra.py +++ b/python/pyarrow/tests/interchange/test_extra.py @@ -21,12 +21,12 @@ from pyarrow.vendored.version import Version import pytest +import pyarrow.interchange as pi from pyarrow.interchange.column import ( _PyArrowColumn, ColumnNullType, DtypeKind, ) -from pyarrow.interchange.from_dataframe import from_dataframe try: import pandas as pd @@ -106,7 +106,7 @@ def test_categorical_roundtrip(): ) pandas_df = table.to_pandas() - result = from_dataframe(pandas_df) + result = pi.from_dataframe(pandas_df) # Checking equality for the values # As the dtype of the indices is changed from int32 in pa.Table @@ -157,7 +157,7 @@ def test_pandas_roundtrip(): ) pandas_df = table.to_pandas() - result = from_dataframe(pandas_df) + result = pi.from_dataframe(pandas_df) assert table.equals(result) table_protocol = table.__dataframe__() From b11d84ee9f0da94842e876a1a44d6dc13d37ac93 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Mon, 19 Dec 2022 08:51:05 +0100 Subject: [PATCH 14/49] Add lost whitespace lines --- python/pyarrow/interchange/buffer.py | 4 ++++ python/pyarrow/interchange/column.py | 22 ++++++++++++++++++++ python/pyarrow/interchange/dataframe.py | 4 ++++ python/pyarrow/interchange/from_dataframe.py | 3 +++ 4 files changed, 33 insertions(+) diff --git a/python/pyarrow/interchange/buffer.py b/python/pyarrow/interchange/buffer.py index 9f30f2b99e3..1f537798130 100644 --- a/python/pyarrow/interchange/buffer.py +++ b/python/pyarrow/interchange/buffer.py @@ -37,11 +37,13 @@ class DlpackDeviceType(enum.IntEnum): class _PyArrowBuffer: """ Data in the buffer is guaranteed to be contiguous in memory. + Note that there is no dtype attribute present, a buffer can be thought of as simply a block of memory. However, if the column that the buffer is attached to has a dtype that's supported by DLPack and ``__dlpack__`` is implemented, then that dtype information will be contained in the return value from ``__dlpack__``. + This distinction is useful to support both data exchange via DLPack on a buffer and (b) dtypes like variable-length strings which do not have a fixed number of bytes per element. @@ -70,9 +72,11 @@ def ptr(self) -> int: def __dlpack__(self): """ Produce DLPack capsule (see array API standard). + Raises: - TypeError : if the buffer contains unsupported dtypes. - NotImplementedError : if DLPack support is not implemented + Useful to have to connect to array libraries. Support optional because it's not completely trivial to implement for a Python-only library. """ diff --git a/python/pyarrow/interchange/column.py b/python/pyarrow/interchange/column.py index a14eb2f91d7..a26fd469a0b 100644 --- a/python/pyarrow/interchange/column.py +++ b/python/pyarrow/interchange/column.py @@ -40,6 +40,7 @@ class DtypeKind(enum.IntEnum): """ Integer enum for data types. + Attributes ---------- INT : int @@ -91,6 +92,7 @@ class DtypeKind(enum.IntEnum): class ColumnNullType(enum.IntEnum): """ Integer enum for null type representation. + Attributes ---------- NON_NULLABLE : int @@ -157,27 +159,33 @@ class _PyArrowColumn: """ A column object, with only the methods and properties required by the interchange protocol defined. + A column can contain one or more chunks. Each chunk can contain up to three buffers - a data buffer, a mask buffer (depending on null representation), and an offsets buffer (if variable-size binary; e.g., variable-length strings). + TBD: Arrow has a separate "null" dtype, and has no separate mask concept. Instead, it seems to use "children" for both columns with a bit mask, and for nested dtypes. Unclear whether this is elegant or confusing. This design requires checking the null representation explicitly. + The Arrow design requires checking: 1. the ARROW_FLAG_NULLABLE (for sentinel values) 2. if a column has two children, combined with one of those children having a null dtype. + Making the mask concept explicit seems useful. One null dtype would not be enough to cover both bit and byte masks, so that would mean even more checking if we did it the Arrow way. + TBD: there's also the "chunk" concept here, which is implicit in Arrow as multiple buffers per array (= column here). Semantically it may make sense to have both: chunks were meant for example for lazy evaluation of data which doesn't fit in memory, while multiple buffers per column could also come from doing a selection operation on a single contiguous buffer. + Given these concepts, one would expect chunks to be all of the same size (say a 10,000 row dataframe could have 10 chunks of 1,000 rows), while multiple buffers could have data-dependent lengths. Not an issue @@ -186,6 +194,7 @@ class _PyArrowColumn: Are multiple chunks *and* multiple buffers per column necessary for the purposes of this interchange protocol, or must producers either reuse the chunk concept for this or copy the data? + Note: this Column object can only be produced by ``__dataframe__``, so doesn't need its own version or ``__column__`` protocol. """ @@ -214,8 +223,10 @@ def __init__( def size(self) -> int: """ Size of the column, in elements. + Corresponds to DataFrame.num_rows() if column is a single chunk; equal to size of this current chunk otherwise. + Is a method rather than a property because it may cause a (potentially expensive) computation for some dataframe implementations. """ @@ -225,6 +236,7 @@ def size(self) -> int: def offset(self) -> int: """ Offset of first element. + May be > 0 if using chunks; for example for a column with N chunks of equal size M (only the last chunk may be shorter), ``offset = n * M``, ``n = 0 .. N-1``. @@ -236,10 +248,12 @@ def dtype(self) -> Tuple[DtypeKind, int, str, str]: """ Dtype description as a tuple ``(kind, bit-width, format string, endianness)``. + Bit-width : the number of bits as an integer Format string : data type description format string in Apache Arrow C Data Interface format. Endianness : current only native endianness (``=``) is supported + Notes: - Kind specifiers are aligned with DLPack where possible (hence the jump to 20, leave enough room for future extension) @@ -308,7 +322,9 @@ def describe_categorical(self) -> CategoricalDescription: - There are only values in the data buffer. - There is a separate non-categorical Column encoding categorical values. + Raises TypeError if the dtype is not categorical + Returns the dictionary with description on how to interpret the data buffer: - "is_ordered" : bool, whether the ordering of dictionary indices @@ -319,6 +335,7 @@ def describe_categorical(self) -> CategoricalDescription: indices to category values (e.g. an array of cat1, cat2, ...). None if not a dictionary-style categorical. + TBD: are there any other in-memory representations that are needed? """ arr = self._col @@ -339,6 +356,7 @@ def describe_null(self) -> Tuple[ColumnNullType, Any]: """ Return the missing value (or "null") representation the column dtype uses, as a tuple ``(kind, value)``. + Value : if kind is "sentinel value", the actual value. If kind is a bit mask or a byte mask, the value (0 or 1) indicating a missing value. None otherwise. @@ -355,6 +373,7 @@ def describe_null(self) -> Tuple[ColumnNullType, Any]: def null_count(self) -> int: """ Number of null elements, if known. + Note: Arrow uses -1 to indicate "unknown", but None seems cleaner. """ return self._col.null_count @@ -377,6 +396,7 @@ def get_chunks( ) -> Iterable[_PyArrowColumn]: """ Return an iterator yielding the chunks. + See `DataFrame.get_chunks` for details on ``n_chunks``. """ if n_chunks and n_chunks > 1: @@ -403,7 +423,9 @@ def get_chunks( def get_buffers(self) -> ColumnBuffers: """ Return a dictionary containing the underlying buffers. + The returned dictionary has the following contents: + - "data": a two-element tuple whose first element is a buffer containing the data and whose second element is the data buffer's associated dtype. diff --git a/python/pyarrow/interchange/dataframe.py b/python/pyarrow/interchange/dataframe.py index fdaf3849546..2a6457718e0 100644 --- a/python/pyarrow/interchange/dataframe.py +++ b/python/pyarrow/interchange/dataframe.py @@ -32,9 +32,11 @@ class _PyArrowDataFrame: """ A data frame class, with only the methods required by the interchange protocol defined. + A "data frame" represents an ordered collection of named columns. A column's "name" must be a unique string. Columns may be accessed by name or by position. + This could be a public data frame class, or an object with the methods and attributes defined on this DataFrame class could be returned from the ``__dataframe__`` method of a public data frame class in a library adhering @@ -163,10 +165,12 @@ def get_chunks( ) -> Iterable[_PyArrowDataFrame]: """ Return an iterator yielding the chunks. + By default (None), yields the chunks that the data is stored as by the producer. If given, ``n_chunks`` must be a multiple of ``self.num_chunks()``, meaning the producer must subdivide each chunk before yielding it. + Note that the producer must ensure that all columns are chunked the same way. """ diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index 05203ff08c3..fa0da200c91 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -72,6 +72,7 @@ def from_dataframe(df: DataFrameObject, allow_copy=True) -> pa.Table: allow_copy : bool, default: True Whether to allow copying the memory to perform the conversion (if false then zero-copy approach is requested). + Returns ------- pa.Table @@ -88,6 +89,7 @@ def from_dataframe(df: DataFrameObject, allow_copy=True) -> pa.Table: def _from_dataframe(df: DataFrameObject, allow_copy=True): """ Build a ``pa.Table`` from the DataFrame interchange object. + Parameters ---------- df : DataFrameObject @@ -96,6 +98,7 @@ def _from_dataframe(df: DataFrameObject, allow_copy=True): allow_copy : bool, default: True Whether to allow copying the memory to perform the conversion (if false then zero-copy approach is requested). + Returns ------- pa.Table From 6c9dce43b5a8394abca19e87aecf6743fd6d53a9 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Mon, 19 Dec 2022 08:55:56 +0100 Subject: [PATCH 15/49] Revert commented categories in CategoricalDescription, column.py --- python/pyarrow/interchange/column.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyarrow/interchange/column.py b/python/pyarrow/interchange/column.py index a26fd469a0b..a643b1a97db 100644 --- a/python/pyarrow/interchange/column.py +++ b/python/pyarrow/interchange/column.py @@ -139,7 +139,7 @@ class CategoricalDescription(TypedDict): is_dictionary: bool # Python-level only (e.g. ``{int: str}``). # None if not a dictionary-style categorical. - # categories: Optional[Column] + categories: Optional[_PyArrowColumn] class Endianness: From 8d91b672bf65fd15d52469e487c432ee6a5fc044 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Mon, 19 Dec 2022 09:08:28 +0100 Subject: [PATCH 16/49] Add _dtype attribute to __inti__ of the Column class and move all the dtype logic into _dtype_from_arrowdtype helper --- python/pyarrow/interchange/column.py | 47 ++++++++++++++-------------- 1 file changed, 24 insertions(+), 23 deletions(-) diff --git a/python/pyarrow/interchange/column.py b/python/pyarrow/interchange/column.py index a643b1a97db..a4a2f322ce1 100644 --- a/python/pyarrow/interchange/column.py +++ b/python/pyarrow/interchange/column.py @@ -219,6 +219,13 @@ def __init__( self._col = column self._allow_copy = allow_copy + dtype = self._col.type + try: + bit_width = dtype.bit_width + except ValueError: # in case of a variable-length strings + bit_width = 8 + self._dtype = self._dtype_from_arrowdtype(dtype, bit_width) + @property def size(self) -> int: """ @@ -273,11 +280,18 @@ def dtype(self) -> Tuple[DtypeKind, int, str, str]: - Data types not included: complex, Arrow-style null, binary, decimal, and nested (list, struct, map, union) dtypes. """ - dtype = self._col.type - try: - bit_width = dtype.bit_width - except ValueError: # in case of a variable-length strings - bit_width = 8 + return self._dtype + + def _dtype_from_arrowdtype( + self, dtype: pa.DataType, bit_width: int + ) -> Tuple[DtypeKind, int, str, str]: + """ + See `self.dtype` for details. + """ + # Note: 'c' (complex) not handled yet (not in array spec v1). + # 'b', 'B' (bytes), 'S', 'a', (old-style string) 'V' (void) + # not handled datetime and timedelta both map to datetime + # (is timedelta handled?) if pa.types.is_large_string(dtype): # format string needs to be changed from "U" to "u" @@ -295,25 +309,12 @@ def dtype(self) -> Tuple[DtypeKind, int, str, str]: f_string = "L" return kind, bit_width, f_string, Endianness.NATIVE else: - return self._dtype_from_arrowdtype(dtype, bit_width) - - def _dtype_from_arrowdtype( - self, dtype, bit_width - ) -> Tuple[DtypeKind, int, str, str]: - """ - See `self.dtype` for details. - """ - # Note: 'c' (complex) not handled yet (not in array spec v1). - # 'b', 'B' (bytes), 'S', 'a', (old-style string) 'V' (void) - # not handled datetime and timedelta both map to datetime - # (is timedelta handled?) + kind, f_string = _PYARROW_KINDS.get(dtype, (None, None)) + if kind is None: + raise ValueError( + f"Data type {dtype} not supported by interchange protocol") - kind, f_string = _PYARROW_KINDS.get(dtype, (None, None)) - if kind is None: - raise ValueError( - f"Data type {dtype} not supported by interchange protocol") - - return kind, bit_width, f_string, Endianness.NATIVE + return kind, bit_width, f_string, Endianness.NATIVE @property def describe_categorical(self) -> CategoricalDescription: From 4643f9b5960fadb660d0426c3853b079892c4505 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Mon, 19 Dec 2022 09:14:53 +0100 Subject: [PATCH 17/49] Raise an error if nan_as_null=True --- python/pyarrow/interchange/dataframe.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/python/pyarrow/interchange/dataframe.py b/python/pyarrow/interchange/dataframe.py index 2a6457718e0..66fbfe94f56 100644 --- a/python/pyarrow/interchange/dataframe.py +++ b/python/pyarrow/interchange/dataframe.py @@ -56,6 +56,11 @@ def __init__( # ``NaT``). This currently has no effect; once support for nullable # extension dtypes is added, this value should be propagated to # columns. + if nan_as_null == True: + raise RuntimeError( + "nan_as_null=True currently has no effect, " + "use the default nan_as_null=False" + ) self._nan_as_null = nan_as_null self._allow_copy = allow_copy From a93a46ec84f498188b46d811c840de443f8d0fa0 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Mon, 19 Dec 2022 09:17:09 +0100 Subject: [PATCH 18/49] Linter corrections --- python/pyarrow/interchange/__init__.py | 2 ++ python/pyarrow/interchange/dataframe.py | 8 ++++---- python/pyarrow/tests/interchange/test_extra.py | 2 +- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/python/pyarrow/interchange/__init__.py b/python/pyarrow/interchange/__init__.py index 685556171ce..7ebe59b499c 100644 --- a/python/pyarrow/interchange/__init__.py +++ b/python/pyarrow/interchange/__init__.py @@ -15,4 +15,6 @@ # specific language governing permissions and limitations # under the License. +# flake8: noqa + from .from_dataframe import from_dataframe diff --git a/python/pyarrow/interchange/dataframe.py b/python/pyarrow/interchange/dataframe.py index 66fbfe94f56..4c0eabd6b93 100644 --- a/python/pyarrow/interchange/dataframe.py +++ b/python/pyarrow/interchange/dataframe.py @@ -56,11 +56,11 @@ def __init__( # ``NaT``). This currently has no effect; once support for nullable # extension dtypes is added, this value should be propagated to # columns. - if nan_as_null == True: + if nan_as_null is True: raise RuntimeError( - "nan_as_null=True currently has no effect, " - "use the default nan_as_null=False" - ) + "nan_as_null=True currently has no effect, " + "use the default nan_as_null=False" + ) self._nan_as_null = nan_as_null self._allow_copy = allow_copy diff --git a/python/pyarrow/tests/interchange/test_extra.py b/python/pyarrow/tests/interchange/test_extra.py index ec0f7e0015a..a9bbfaa240f 100644 --- a/python/pyarrow/tests/interchange/test_extra.py +++ b/python/pyarrow/tests/interchange/test_extra.py @@ -21,7 +21,7 @@ from pyarrow.vendored.version import Version import pytest -import pyarrow.interchange as pi +import pyarrow.interchange as pi from pyarrow.interchange.column import ( _PyArrowColumn, ColumnNullType, From 0b231ea87c5f11b798c0daf8444ef042366986ac Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Mon, 19 Dec 2022 11:28:06 +0100 Subject: [PATCH 19/49] Add better test coverage for test_mixed_dtypes and test_dtypes --- .../interchange/test_interchange_spec.py | 99 ++++++++++++------- 1 file changed, 66 insertions(+), 33 deletions(-) diff --git a/python/pyarrow/tests/interchange/test_interchange_spec.py b/python/pyarrow/tests/interchange/test_interchange_spec.py index 09fcb9c0f09..75fd8a8450f 100644 --- a/python/pyarrow/tests/interchange/test_interchange_spec.py +++ b/python/pyarrow/tests/interchange/test_interchange_spec.py @@ -16,63 +16,96 @@ # under the License. import ctypes +import hypothesis as h +import hypothesis.strategies as st +import numpy as np import pyarrow as pa +import pyarrow.tests.strategies as past import pytest -@pytest.mark.parametrize( - "test_data", - [ - {"a": ["foo", "bar"], "b": ["baz", "qux"]}, - {"a": [1.5, 2.5, 3.5], "b": [9.2, 10.5, 11.8]}, - {"A": [1, 2, 3, 4], "B": [1, 2, 3, 4]}, - ], +all_types = st.deferred( + lambda: ( + past.signed_integer_types | + past.unsigned_integer_types | + past.floating_types | + past.bool_type | + past.string_type | + past.large_string_type + ) ) -def test_only_one_dtype(test_data): - columns = list(test_data.keys()) - table = pa.table(test_data) + + +@h.given(past.arrays(all_types, size=3)) +def test_dtypes(arr): + table = pa.table([arr], names=["a"]) df = table.__dataframe__() - column_size = len(test_data[columns[0]]) - for column in columns: - null_count = df.get_column_by_name(column).null_count - assert null_count == 0 - assert isinstance(null_count, int) - assert df.get_column_by_name(column).size == column_size - assert df.get_column_by_name(column).offset == 0 + null_count = df.get_column(0).null_count + assert null_count == arr.null_count + assert isinstance(null_count, int) + assert df.get_column(0).size == 3 + assert df.get_column(0).offset == 0 -def test_mixed_dtypes(): +@pytest.mark.parametrize( + "uint, uint_bw", + [ + (pa.uint8(), 8), + (pa.uint16(), 16), + (pa.uint32(), 32) + ] +) +@pytest.mark.parametrize( + "int, int_bw", [ + (pa.int8(), 8), + (pa.int16(), 16), + (pa.int32(), 32), + (pa.int64(), 64) + ] +) +@pytest.mark.parametrize( + "float, float_bw, np_float", [ + (pa.float16(), 16, np.float16), + (pa.float32(), 32, np.float32), + (pa.float64(), 64, np.float64) + ] +) +@pytest.mark.parametrize("unit", ['s', 'ms', 'us', 'ns']) +@pytest.mark.parametrize("tz", ['', 'America/New_York', '+07:30', '-04:30']) +def test_mixed_dtypes(uint, uint_bw, int, int_bw, + float, float_bw, np_float, unit, tz): from datetime import datetime as dt + arr = [1, 2, 3] + dt_arr = [dt(2007, 7, 13), dt(2007, 7, 14), dt(2007, 7, 15)] table = pa.table( { - "a": [1, 2, 3], # dtype kind INT = 0 - "b": [3, 4, 5], # dtype kind INT = 0 - "c": [1.5, 2.5, 3.5], # dtype kind FLOAT = 2 - "d": [9, 10, 11], # dtype kind INT = 0 - "e": [True, False, True], # dtype kind BOOLEAN = 20 - "f": ["a", "", "c"], # dtype kind STRING = 21 - "g": [dt(2007, 7, 13), dt(2007, 7, 14), - dt(2007, 7, 15)] # dtype kind DATETIME = 22 + "a": pa.array(arr, type=uint), + "b": pa.array(arr, type=int), + "c": pa.array(np.array(arr, dtype=np_float), type=float), + "d": [True, False, True], + "e": ["a", "", "c"], + "f": pa.array(dt_arr, type=pa.timestamp(unit, tz=tz)) } ) df = table.__dataframe__() - # for meanings of dtype[0] see the spec; we cannot import the - # spec here as this file is expected to be vendored *anywhere*; - # values for dtype[0] are explained above - columns = {"a": 0, "b": 0, "c": 2, "d": 0, "e": 20, "f": 21, "g": 22} + # 0 = DtypeKind.INT, 1 = DtypeKind.UINT, 2 = DtypeKind.FLOAT, + # 20 = DtypeKind.BOOL, 21 = DtypeKind.STRING, 22 = DtypeKind.DATETIME + # see DtypeKind class in column.py + columns = {"a": 1, "b": 0, "c": 2, "d": 20, "e": 21, "f": 22} for column, kind in columns.items(): col = df.get_column_by_name(column) + assert col.null_count == 0 - assert isinstance(col.null_count, int) assert col.size == 3 assert col.offset == 0 - assert col.dtype[0] == kind - assert df.get_column_by_name("c").dtype[1] == 64 + assert df.get_column_by_name("a").dtype[1] == uint_bw + assert df.get_column_by_name("b").dtype[1] == int_bw + assert df.get_column_by_name("c").dtype[1] == float_bw def test_na_float(): From d8ab9024b83427bc9ad635f58d03dfd0bfdb1eda Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Mon, 19 Dec 2022 12:47:10 +0100 Subject: [PATCH 20/49] Add better test coverage for test_pandas_roundtrip and add large_memory mark to test_large_string --- .../pyarrow/tests/interchange/test_extra.py | 67 +++++++------------ .../interchange/test_interchange_spec.py | 2 + 2 files changed, 28 insertions(+), 41 deletions(-) diff --git a/python/pyarrow/tests/interchange/test_extra.py b/python/pyarrow/tests/interchange/test_extra.py index a9bbfaa240f..ce0d87604a5 100644 --- a/python/pyarrow/tests/interchange/test_extra.py +++ b/python/pyarrow/tests/interchange/test_extra.py @@ -41,6 +41,7 @@ def test_datetime(): col = table.__dataframe__().get_column_by_name("A") assert col.size == 2 + assert col.offset == 0 assert col.null_count == 1 assert col.dtype[0] == DtypeKind.DATETIME assert col.describe_null == (ColumnNullType.USE_BITMASK, 0) @@ -138,21 +139,36 @@ def test_categorical_roundtrip(): @pytest.mark.pandas -def test_pandas_roundtrip(): +@pytest.mark.parametrize( + "uint", [pa.uint8(), pa.uint16(), pa.uint32()] +) +@pytest.mark.parametrize( + "int", [pa.int8(), pa.int16(), pa.int32(), pa.int64()] +) +@pytest.mark.parametrize( + "float, np_float", [ + (pa.float16(), np.float16), + (pa.float32(), np.float32), + (pa.float64(), np.float64) + ] +) +def test_pandas_roundtrip(uint, int, float, np_float): if Version(pd.__version__) < Version("1.5.0"): pytest.skip("__dataframe__ added to pandas in 1.5.0") from datetime import datetime as dt + arr = [1, 2, 3] + dt_arr = [dt(2007, 7, 13), dt(2007, 7, 14), dt(2007, 7, 15)] table = pa.table( { - "a": [1, 2, 3, 4], # dtype kind INT = 0 - "b": [3, 4, 5, 6], # dtype kind INT = 0 - "c": [1.5, 2.5, 3.5, 4.5], # dtype kind FLOAT = 2 - "d": [9, 10, 11, 12], # dtype kind INT = 0 - "e": [True, True, False, False], # dtype kind BOOLEAN = 20 - "f": ["a", "", "c", "d"], # dtype kind STRING = 21 - "g": [dt(2007, 7, 13), dt(2007, 7, 14), - dt(2007, 7, 15), dt(2007, 7, 16)] # dtype kind DATETIME = 22 + "a": pa.array(arr, type=uint), + "b": pa.array(arr, type=int), + "c": pa.array(np.array(arr, dtype=np_float), type=float), + "d": [True, False, True], + "e": ["a", "", "c"], + # pandas seems to only support datetime64 in "us" resolution(?) + # without time zone + "f": pa.array(dt_arr, type=pa.timestamp('us')) } ) @@ -170,6 +186,7 @@ def test_pandas_roundtrip(): @pytest.mark.pandas +@pytest.mark.large_memory def test_large_string(): if Version(pd.__version__) < Version("1.5.0"): pytest.skip("__dataframe__ added to pandas in 1.5.0") @@ -185,35 +202,3 @@ def test_large_string(): assert result["large_string"][0] == data[0].decode() assert result.size == 3*1024**2 - - -def test_uint(): - arr = pa.array([1, 2, 3, None], type=pa.uint16()) - table = pa.table([arr], names=["uint"]) - df = table.__dataframe__() - col = df.get_column(0) - - assert col.null_count == 1 - assert isinstance(col.null_count, int) - assert col.size == 4 - assert col.offset == 0 - - assert col.dtype[0] == DtypeKind.UINT - assert col.dtype[1] == 16 - - -@pytest.mark.pandas -def test_uint_roundtrip(): - if Version(pd.__version__) < Version("1.5.0"): - pytest.skip("__dataframe__ added to pandas in 1.5.0") - - from pandas.core.interchange.from_dataframe import ( - from_dataframe as pandas_from_dataframe - ) - - arr = pa.array([1, 2, 3, 4], type=pa.uint16()) - table = pa.table([arr], names=["uint"]) - result = pandas_from_dataframe(table) - result_to_table = pa.Table.from_pandas(result) - - assert table[0] == result_to_table[0] diff --git a/python/pyarrow/tests/interchange/test_interchange_spec.py b/python/pyarrow/tests/interchange/test_interchange_spec.py index 75fd8a8450f..4d4c3fd9b13 100644 --- a/python/pyarrow/tests/interchange/test_interchange_spec.py +++ b/python/pyarrow/tests/interchange/test_interchange_spec.py @@ -37,6 +37,8 @@ ) +# datetime is tested in test_extra.py +# dictionary is tested in test_categorical() @h.given(past.arrays(all_types, size=3)) def test_dtypes(arr): table = pa.table([arr], names=["a"]) From 21af8fbb2afe368d59dab56f0a3a1d2e8bc86998 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Tue, 20 Dec 2022 09:33:51 +0100 Subject: [PATCH 21/49] Add pyarrow roundtrip tests and make additional corrections to the code, skip some of pandas roundtrip tests --- python/pyarrow/interchange/column.py | 18 +- python/pyarrow/interchange/from_dataframe.py | 7 +- .../pyarrow/tests/interchange/test_extra.py | 195 +++++++++++++++--- .../interchange/test_interchange_spec.py | 59 ++++-- 4 files changed, 209 insertions(+), 70 deletions(-) diff --git a/python/pyarrow/interchange/column.py b/python/pyarrow/interchange/column.py index a4a2f322ce1..8621a1b5ae4 100644 --- a/python/pyarrow/interchange/column.py +++ b/python/pyarrow/interchange/column.py @@ -226,7 +226,6 @@ def __init__( bit_width = 8 self._dtype = self._dtype_from_arrowdtype(dtype, bit_width) - @property def size(self) -> int: """ Size of the column, in elements. @@ -293,12 +292,7 @@ def _dtype_from_arrowdtype( # not handled datetime and timedelta both map to datetime # (is timedelta handled?) - if pa.types.is_large_string(dtype): - # format string needs to be changed from "U" to "u" - # in case of large strings (make an issue in pandas?) - kind = DtypeKind.STRING - return kind, bit_width, "u", Endianness.NATIVE - elif pa.types.is_timestamp(dtype): + if pa.types.is_timestamp(dtype): kind = DtypeKind.DATETIME ts = dtype.unit[0] tz = dtype.tz if dtype.tz else "" @@ -401,8 +395,8 @@ def get_chunks( See `DataFrame.get_chunks` for details on ``n_chunks``. """ if n_chunks and n_chunks > 1: - chunk_size = self.size // n_chunks - if self.size % n_chunks != 0: + chunk_size = self.size() // n_chunks + if self.size() % n_chunks != 0: chunk_size += 1 array = self._col @@ -412,12 +406,6 @@ def get_chunks( array.slice(start, chunk_size), self._allow_copy ) i += 1 - - elif isinstance(self._col, pa.ChunkedArray): - return [ - _PyArrowColumn(chunk, self._allow_copy) - for chunk in self._col.chunks - ] else: yield self diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index fa0da200c91..33a8b456f2d 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -368,8 +368,6 @@ def parse_datetime_format_str(format_str): timestamp_meta = re.match(r"ts([smun]):(.*)", format_str) if timestamp_meta: unit, tz = timestamp_meta.group(1), timestamp_meta.group(2) - if tz != "": - raise NotImplementedError("Timezones are not supported yet") if unit != "s": # the format string describes only a first letter of the unit, so # add one extra letter to convert the unit to numpy-style: @@ -378,9 +376,6 @@ def parse_datetime_format_str(format_str): return unit, tz - # TODO - # date 'td{Days/Ms}' - raise NotImplementedError(f"DateTime kind is not supported: {format_str}") @@ -452,7 +447,7 @@ def buffers_to_array( # and add it to the construction of an array offset_pa_buffer = pa.py_buffer(offset_buff._x) array = pa.Array.from_buffers( - pa.large_string(), + data_dtype, length, [validity_pa_buff, offset_pa_buffer, data_pa_buffer], offset=offset, diff --git a/python/pyarrow/tests/interchange/test_extra.py b/python/pyarrow/tests/interchange/test_extra.py index ce0d87604a5..88a74bef232 100644 --- a/python/pyarrow/tests/interchange/test_extra.py +++ b/python/pyarrow/tests/interchange/test_extra.py @@ -27,6 +27,7 @@ ColumnNullType, DtypeKind, ) +from pyarrow.interchange.from_dataframe import _from_dataframe try: import pandas as pd @@ -35,12 +36,14 @@ pass -def test_datetime(): - df = pd.DataFrame({"A": [dt(2007, 7, 13), None]}) - table = pa.table(df) +@pytest.mark.parametrize("unit", ['s', 'ms', 'us', 'ns']) +@pytest.mark.parametrize("tz", ['', 'America/New_York', '+07:30', '-04:30']) +def test_datetime(unit, tz): + dt_arr = [dt(2007, 7, 13), dt(2007, 7, 14), None] + table = pa.table({"A": pa.array(dt_arr, type=pa.timestamp(unit, tz=tz))}) col = table.__dataframe__().get_column_by_name("A") - assert col.size == 2 + assert col.size() == 3 assert col.offset == 0 assert col.null_count == 1 assert col.dtype[0] == DtypeKind.DATETIME @@ -60,7 +63,7 @@ def test_array_to_pyarrowcolumn(test_data, kind): arr_column = _PyArrowColumn(arr) assert arr_column._col == arr - assert arr_column.size == len(test_data) + assert arr_column.size() == len(test_data) assert arr_column.dtype[0] == kind assert arr_column.num_chunks() == 1 assert arr_column.null_count == 0 @@ -71,25 +74,24 @@ def test_array_to_pyarrowcolumn(test_data, kind): assert chunk == arr_column -@pytest.mark.pandas def test_offset_of_sliced_array(): - # if Version(pd.__version__) < Version("1.5.0"): - # pytest.skip("__dataframe__ added to pandas in 1.5.0") - - # from pandas.core.interchange.from_dataframe import ( - # from_dataframe as pandas_from_dataframe - # ) - arr = pa.array([1, 2, 3, 4]) arr_sliced = arr.slice(2, 2) - # table = pa.table([arr], names=["arr"]) + table = pa.table([arr], names=["arr"]) table_sliced = pa.table([arr_sliced], names=["arr_sliced"]) col = table_sliced.__dataframe__().get_column(0) assert col.offset == 2 - # TODO: Investigate why this part is not working currently! + result = _from_dataframe(table_sliced.__dataframe__()) + assert table_sliced.equals(result) + assert not table.equals(result) + + # pandas hardcodes offset to 0: + # https://github.com/pandas-dev/pandas/blob/5c66e65d7b9fef47ccb585ce2fd0b3ea18dc82ea/pandas/core/interchange/from_dataframe.py#L247 + # so conversion to pandas can't be tested currently + # df = pandas_from_dataframe(table) # df_sliced = pandas_from_dataframe(table_sliced) @@ -97,8 +99,13 @@ def test_offset_of_sliced_array(): # check_index=False, check_names=False) +# Currently errors due to string conversion +# as col.size is called as a property not method in pandas +# see L255-L257 in pandas/core/interchange/from_dataframe.py @pytest.mark.pandas def test_categorical_roundtrip(): + pytest.skip("Bug in pandas implementation") + if Version(pd.__version__) < Version("1.5.0"): pytest.skip("__dataframe__ added to pandas in 1.5.0") arr = ["Mon", "Tue", "Mon", "Wed", "Mon", "Thu", "Fri", "Sat", "Sun"] @@ -147,7 +154,7 @@ def test_categorical_roundtrip(): ) @pytest.mark.parametrize( "float, np_float", [ - (pa.float16(), np.float16), + # (pa.float16(), np.float16), #not supported by pandas (pa.float32(), np.float32), (pa.float64(), np.float64) ] @@ -155,24 +162,25 @@ def test_categorical_roundtrip(): def test_pandas_roundtrip(uint, int, float, np_float): if Version(pd.__version__) < Version("1.5.0"): pytest.skip("__dataframe__ added to pandas in 1.5.0") - from datetime import datetime as dt arr = [1, 2, 3] - dt_arr = [dt(2007, 7, 13), dt(2007, 7, 14), dt(2007, 7, 15)] table = pa.table( { "a": pa.array(arr, type=uint), "b": pa.array(arr, type=int), "c": pa.array(np.array(arr, dtype=np_float), type=float), - "d": [True, False, True], - "e": ["a", "", "c"], - # pandas seems to only support datetime64 in "us" resolution(?) - # without time zone - "f": pa.array(dt_arr, type=pa.timestamp('us')) + # String dtype errors as col.size is called as a property + # not method in pandas, see L255-L257 in + # pandas/core/interchange/from_dataframe.py + # "d": ["a", "", "c"], + # large string is not supported by pandas implementation } ) - pandas_df = table.to_pandas() + from pandas.core.interchange.from_dataframe import ( + from_dataframe as pandas_from_dataframe + ) + pandas_df = pandas_from_dataframe(table) result = pi.from_dataframe(pandas_df) assert table.equals(result) @@ -185,20 +193,141 @@ def test_pandas_roundtrip(uint, int, float, np_float): assert table_protocol.column_names() == result_protocol.column_names() -@pytest.mark.pandas -@pytest.mark.large_memory -def test_large_string(): +def test_roundtrip_pandas_boolean(): + # Boolean pyarrow Arrays are casted to uint8 as bit packed boolean + # is not yet supported by the protocol + if Version(pd.__version__) < Version("1.5.0"): pytest.skip("__dataframe__ added to pandas in 1.5.0") + table = pa.table({"a": [True, False, True]}) + expected = pa.table({"a": pa.array([1, 0, 1], type=pa.uint8())}) + from pandas.core.interchange.from_dataframe import ( from_dataframe as pandas_from_dataframe ) + pandas_df = pandas_from_dataframe(table) + result = pi.from_dataframe(pandas_df) + + assert expected.equals(result) + + expected_protocol = expected.__dataframe__() + result_protocol = result.__dataframe__() + + assert expected_protocol.num_columns() == result_protocol.num_columns() + assert expected_protocol.num_rows() == result_protocol.num_rows() + assert expected_protocol.num_chunks() == result_protocol.num_chunks() + assert expected_protocol.column_names() == result_protocol.column_names() + + +@pytest.mark.parametrize("unit", ['s', 'ms', 'us', 'ns']) +def test_roundtrip_pandas_datetime(unit): + # pandas always creates datetime64 in "us" + # resolution, timezones are not yet supported + + if Version(pd.__version__) < Version("1.5.0"): + pytest.skip("__dataframe__ added to pandas in 1.5.0") + from datetime import datetime as dt + + dt_arr = [dt(2007, 7, 13), dt(2007, 7, 14), dt(2007, 7, 15)] + table = pa.table({"a": pa.array(dt_arr, type=pa.timestamp(unit))}) + expected = pa.table({"a": pa.array(dt_arr, type=pa.timestamp('us'))}) + + from pandas.core.interchange.from_dataframe import ( + from_dataframe as pandas_from_dataframe + ) + pandas_df = pandas_from_dataframe(table) + result = pi.from_dataframe(pandas_df) + + assert expected.equals(result) + + expected_protocol = expected.__dataframe__() + result_protocol = result.__dataframe__() + + assert expected_protocol.num_columns() == result_protocol.num_columns() + assert expected_protocol.num_rows() == result_protocol.num_rows() + assert expected_protocol.num_chunks() == result_protocol.num_chunks() + assert expected_protocol.column_names() == result_protocol.column_names() + + +@pytest.mark.parametrize( + "uint", [pa.uint8(), pa.uint16(), pa.uint32()] +) +@pytest.mark.parametrize( + "int", [pa.int8(), pa.int16(), pa.int32(), pa.int64()] +) +@pytest.mark.parametrize( + "float, np_float", [ + (pa.float16(), np.float16), + # (pa.float32(), np.float32), #errors due to inequality in nan + # (pa.float64(), np.float64) #errors due to inequality in nan + ] +) +@pytest.mark.parametrize("unit", ['s', 'ms', 'us', 'ns']) +@pytest.mark.parametrize("tz", ['America/New_York', '+07:30', '-04:30']) +def test_pyarrow_roundtrip(uint, int, float, np_float, unit, tz): + + from datetime import datetime as dt + arr = [1, 2, None] + dt_arr = [dt(2007, 7, 13), None, dt(2007, 7, 15)] + + table = pa.table( + { + "a": pa.array(arr, type=uint), + "b": pa.array(arr, type=int), + "c": pa.array(np.array(arr, dtype=np_float), type=float), + "d": [True, False, True], + "e": ["a", "", "c"], + "f": pa.array(dt_arr, type=pa.timestamp(unit, tz=tz)) + } + ) + result = _from_dataframe(table.__dataframe__()) + + assert table.equals(result) + + table_protocol = table.__dataframe__() + result_protocol = result.__dataframe__() + + assert table_protocol.num_columns() == result_protocol.num_columns() + assert table_protocol.num_rows() == result_protocol.num_rows() + assert table_protocol.num_chunks() == result_protocol.num_chunks() + assert table_protocol.column_names() == result_protocol.column_names() + + +def test_pyarrow_roundtrip_categorical(): + arr = ["Mon", "Tue", "Mon", "Wed", "Mon", "Thu", "Fri", "Sat", "Sun"] + table = pa.table( + {"weekday": pa.array(arr).dictionary_encode()} + ) + result = _from_dataframe(table.__dataframe__()) + + assert table.equals(result) + + table_protocol = table.__dataframe__() + result_protocol = result.__dataframe__() + + assert table_protocol.num_columns() == result_protocol.num_columns() + assert table_protocol.num_rows() == result_protocol.num_rows() + assert table_protocol.num_chunks() == result_protocol.num_chunks() + assert table_protocol.column_names() == result_protocol.column_names() + + col_table = table_protocol.get_column(0) + col_result = result_protocol.get_column(0) + + assert col_result.dtype[0] == DtypeKind.CATEGORICAL + assert col_result.dtype[0] == col_table.dtype[0] + assert col_result.size() == col_table.size() + assert col_result.offset == col_table.offset + + desc_cat_table = col_result.describe_categorical + desc_cat_result = col_result.describe_categorical + + assert desc_cat_table["is_ordered"] == desc_cat_result["is_ordered"] + assert desc_cat_table["is_dictionary"] == desc_cat_result["is_dictionary"] + assert isinstance(desc_cat_result["categories"]._col, pa.Array) - data = np.array([b'x'*1024]*(3*1024**2), dtype='object') # 3GB bytes data - arr = pa.array(data, type=pa.large_string()) - table = pa.table([arr], names=["large_string"]) - result = pandas_from_dataframe(table) - assert result["large_string"][0] == data[0].decode() - assert result.size == 3*1024**2 +def test_nan_as_null(): + table = pa.table({"a": [1, 2, 3, 4]}) + with pytest.raises(RuntimeError): + table.__dataframe__(nan_as_null=True) diff --git a/python/pyarrow/tests/interchange/test_interchange_spec.py b/python/pyarrow/tests/interchange/test_interchange_spec.py index 4d4c3fd9b13..42ec8053599 100644 --- a/python/pyarrow/tests/interchange/test_interchange_spec.py +++ b/python/pyarrow/tests/interchange/test_interchange_spec.py @@ -47,7 +47,7 @@ def test_dtypes(arr): null_count = df.get_column(0).null_count assert null_count == arr.null_count assert isinstance(null_count, int) - assert df.get_column(0).size == 3 + assert df.get_column(0).size() == 3 assert df.get_column(0).offset == 0 @@ -101,7 +101,7 @@ def test_mixed_dtypes(uint, uint_bw, int, int_bw, col = df.get_column_by_name(column) assert col.null_count == 0 - assert col.size == 3 + assert col.size() == 3 assert col.offset == 0 assert col.dtype[0] == kind @@ -128,7 +128,7 @@ def test_noncategorical(): def test_categorical(): import pyarrow as pa - arr = ["Mon", "Tue", "Mon", "Wed", "Mon", "Thu", "Fri", "Sat", "Sun"] + arr = ["Mon", "Tue", "Mon", "Wed", "Mon", "Thu", "Fri", "Sat", None] table = pa.table( {"weekday": pa.array(arr).dictionary_encode()} ) @@ -170,24 +170,51 @@ def test_column_get_chunks(size, n_chunks): df = table.__dataframe__() chunks = list(df.get_column(0).get_chunks(n_chunks)) assert len(chunks) == n_chunks - assert sum(chunk.size for chunk in chunks) == size + assert sum(chunk.size() for chunk in chunks) == size -def test_get_columns(): - table = pa.table({"a": [0, 1], "b": [2.5, 3.5]}) +@pytest.mark.pandas +@pytest.mark.parametrize( + "uint", [pa.uint8(), pa.uint16(), pa.uint32()] +) +@pytest.mark.parametrize( + "int", [pa.int8(), pa.int16(), pa.int32(), pa.int64()] +) +@pytest.mark.parametrize( + "float, np_float", [ + (pa.float16(), np.float16), + (pa.float32(), np.float32), + (pa.float64(), np.float64) + ] +) +def test_get_columns(uint, int, float, np_float): + arr = [[1, 2, 3], [4, 5]] + arr_float = np.array([1, 2, 3, 4, 5], dtype=np_float) + table = pa.table( + { + "a": pa.chunked_array(arr, type=uint), + "b": pa.chunked_array(arr, type=int), + "c": pa.array(arr_float, type=float) + } + ) df = table.__dataframe__() for col in df.get_columns(): - assert col.size == 2 + assert col.size() == 5 assert col.num_chunks() == 1 - # for meanings of dtype[0] see the spec; we cannot import the - # spec here as this file is expected to be vendored *anywhere* - assert df.get_column(0).dtype[0] == 0 # INT - assert df.get_column(1).dtype[0] == 2 # FLOAT + + # 0 = DtypeKind.INT, 1 = DtypeKind.UINT, 2 = DtypeKind.FLOAT, + # see DtypeKind class in column.py + assert df.get_column(0).dtype[0] == 1 # UINT + assert df.get_column(1).dtype[0] == 0 # INT + assert df.get_column(2).dtype[0] == 2 # FLOAT -def test_buffer(): +@pytest.mark.parametrize( + "int", [pa.int8(), pa.int16(), pa.int32(), pa.int64()] +) +def test_buffer(int): arr = [0, 1, -1] - table = pa.table({"a": arr}) + table = pa.table({"a": pa.array(arr, type=int)}) df = table.__dataframe__() col = df.get_column(0) buf = col.get_buffers() @@ -198,9 +225,9 @@ def test_buffer(): assert dataBuf.ptr != 0 device, _ = dataBuf.__dlpack_device__() - # for meanings of dtype[0] see the spec; we cannot import the spec - # here as this file is expected to be vendored *anywhere* - assert dataDtype[0] == 0 # INT + # 0 = DtypeKind.INT + # see DtypeKind class in column.py + assert dataDtype[0] == 0 if device == 1: # CPU-only as we're going to directly read memory here bitwidth = dataDtype[1] From d6140d4ef24a06c963d65bf1aa9106ae00bb37f2 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Tue, 20 Dec 2022 11:36:38 +0100 Subject: [PATCH 22/49] Correct large string handling and make smaller corrections in convert methods definitions --- python/pyarrow/interchange/column.py | 6 ++- python/pyarrow/interchange/from_dataframe.py | 53 ++++++++++++++----- .../pyarrow/tests/interchange/test_extra.py | 38 +++++++++++++ 3 files changed, 82 insertions(+), 15 deletions(-) diff --git a/python/pyarrow/interchange/column.py b/python/pyarrow/interchange/column.py index 8621a1b5ae4..31860594138 100644 --- a/python/pyarrow/interchange/column.py +++ b/python/pyarrow/interchange/column.py @@ -84,7 +84,7 @@ class DtypeKind(enum.IntEnum): pa.float32(): (DtypeKind.FLOAT, "f"), pa.float64(): (DtypeKind.FLOAT, "g"), pa.bool_(): (DtypeKind.BOOL, "b"), - pa.string(): (DtypeKind.STRING, "u"), # utf-8 + pa.string(): (DtypeKind.STRING, "u"), pa.large_string(): (DtypeKind.STRING, "U"), } @@ -371,7 +371,9 @@ def null_count(self) -> int: Note: Arrow uses -1 to indicate "unknown", but None seems cleaner. """ - return self._col.null_count + arrow_null_count = self._col.null_count + n = arrow_null_count if arrow_null_count != -1 else None + return n @property def metadata(self) -> Dict[str, Any]: diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index 33a8b456f2d..12a10d61ae9 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -179,14 +179,17 @@ def column_to_array( that keeps the memory alive. """ buffers = col.get_buffers() + null_count = col.null_count data = buffers_to_array(buffers, col.size(), - col.describe_null, col.offset, allow_copy) + col.describe_null, + col.dtype, col.offset, + allow_copy, null_count) return data, buffers def bool_8_column_to_array( col: ColumnObject, - allow_copy=True + allow_copy: bool = True ) -> tuple[pa.Array, Any]: """ Convert a column holding boolean dtype with bit width = 8 to a @@ -247,7 +250,7 @@ def bool_8_column_to_array( def categorical_column_to_dictionary( col: ColumnObject, - allow_copy: bool = False + allow_copy: bool = True ) -> tuple[pa.DictionaryArray, Any]: """ Convert a column holding categorical data to a pa.DictionaryArray. @@ -273,8 +276,10 @@ def categorical_column_to_dictionary( dictionary = column_to_array(cat_column)[0] buffers = col.get_buffers() - indices = buffers_to_array( - buffers, col.size(), col.describe_null, col.offset, allow_copy) + indices = buffers_to_array(buffers, col.size(), + col.describe_null, + col.dtype, col.offset, + allow_copy) if null_kind == ColumnNullType.USE_SENTINEL: if not allow_copy: @@ -291,7 +296,7 @@ def categorical_column_to_dictionary( def datetime_column_to_array( col: ColumnObject, - allow_copy: bool = False + allow_copy: bool = True ) -> tuple[pa.Array, Any]: """ Convert a column holding DateTime data to a NumPy array. @@ -383,8 +388,10 @@ def buffers_to_array( buffers: ColumnBuffers, length: int, describe_null: ColumnNullType, + validity_dtype: Dtype, offset: int = 0, - allow_copy: bool = False + allow_copy: bool = True, + null_count: int = -1 ) -> pa.Array: """ Build a PyArrow array from the passed buffer. @@ -399,8 +406,15 @@ def buffers_to_array( describe_null: ColumnNullType Null representation the column dtype uses, as a tuple ``(kind, value)`` + validity_dtype : Dtype, + Dtype description as a tuple ``(kind, bit-width, format string, + endianness)``. offset : int, default: 0 Number of elements to offset from the start of the buffer. + allow_copy : bool, default: True + Whether to allow copying the memory to perform the conversion + (if false then zero-copy approach is requested). + null_count : int, default: -1 Returns ------- @@ -434,6 +448,7 @@ def buffers_to_array( # Construct a validity pyarrow Buffer or a bytemask, if exists validity_pa_buff = validity_buff + _, _, validity_f_string, _ = validity_dtype bytemask = None if validity_pa_buff: validity_pa_buff, bytemask = validity_buffer(validity_buff, @@ -446,12 +461,24 @@ def buffers_to_array( # If an offset buffer exists, construct an offset pyarrow Buffer # and add it to the construction of an array offset_pa_buffer = pa.py_buffer(offset_buff._x) - array = pa.Array.from_buffers( - data_dtype, - length, - [validity_pa_buff, offset_pa_buffer, data_pa_buffer], - offset=offset, - ) + if validity_f_string == 'U': + if not null_count: + null_count = -1 + array = pa.LargeStringArray.from_buffers( + length, + offset_pa_buffer, + data_pa_buffer, + validity_pa_buff, + null_count=null_count, + offset=offset, + ) + else: + array = pa.Array.from_buffers( + pa.string(), + length, + [validity_pa_buff, offset_pa_buffer, data_pa_buffer], + offset=offset, + ) else: array = pa.Array.from_buffers( data_dtype, diff --git a/python/pyarrow/tests/interchange/test_extra.py b/python/pyarrow/tests/interchange/test_extra.py index 88a74bef232..954fe28a3be 100644 --- a/python/pyarrow/tests/interchange/test_extra.py +++ b/python/pyarrow/tests/interchange/test_extra.py @@ -193,6 +193,7 @@ def test_pandas_roundtrip(uint, int, float, np_float): assert table_protocol.column_names() == result_protocol.column_names() +@pytest.mark.pandas def test_roundtrip_pandas_boolean(): # Boolean pyarrow Arrays are casted to uint8 as bit packed boolean # is not yet supported by the protocol @@ -220,6 +221,7 @@ def test_roundtrip_pandas_boolean(): assert expected_protocol.column_names() == result_protocol.column_names() +@pytest.mark.pandas @pytest.mark.parametrize("unit", ['s', 'ms', 'us', 'ns']) def test_roundtrip_pandas_datetime(unit): # pandas always creates datetime64 in "us" @@ -250,6 +252,25 @@ def test_roundtrip_pandas_datetime(unit): assert expected_protocol.column_names() == result_protocol.column_names() +@pytest.mark.large_memory +@pytest.mark.pandas +def test_pandas_assertion_error_large_string(): + # Test AssertionError as pandas does not support "U" type strings + if Version(pd.__version__) < Version("1.5.0"): + pytest.skip("__dataframe__ added to pandas in 1.5.0") + + data = np.array([b'x'*1024]*(3*1024**2), dtype='object') # 3GB bytes data + arr = pa.array(data, type=pa.large_string()) + table = pa.table([arr], names=["large_string"]) + + from pandas.core.interchange.from_dataframe import ( + from_dataframe as pandas_from_dataframe + ) + + with pytest.raises(AssertionError): + pandas_from_dataframe(table) + + @pytest.mark.parametrize( "uint", [pa.uint8(), pa.uint16(), pa.uint32()] ) @@ -327,6 +348,23 @@ def test_pyarrow_roundtrip_categorical(): assert isinstance(desc_cat_result["categories"]._col, pa.Array) +@pytest.mark.large_memory +def test_pyarrow_roundtrip_large_string(): + + data = np.array([b'x'*1024]*(3*1024**2), dtype='object') # 3GB bytes data + arr = pa.array(data, type=pa.large_string()) + table = pa.table([arr], names=["large_string"]) + + result = _from_dataframe(table.__dataframe__()) + col = result.__dataframe__().get_column(0) + + assert col.size() == 3*1024**2 + assert pa.types.is_large_string(table[0].type) + assert pa.types.is_large_string(result[0].type) + + assert table.equals(result) + + def test_nan_as_null(): table = pa.table({"a": [1, 2, 3, 4]}) with pytest.raises(RuntimeError): From e0d1e636e99752bf26da3c566543fcbdd7bcdcbb Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Wed, 21 Dec 2022 20:05:20 +0100 Subject: [PATCH 23/49] Change dict arguments in protocol_df_chunk_to_pyarrow --- python/pyarrow/interchange/from_dataframe.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index 12a10d61ae9..a3e24f9c6d6 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -129,7 +129,7 @@ def protocol_df_chunk_to_pyarrow( """ # We need a dict of columns here, with each column being a PyArrow # or NumPy array. - columns: dict[str, Any] = {} + columns: dict[str, pa.Array | np.ndarray] = {} buffers = [] # hold on to buffers, keeps memory alive for name in df.column_names(): if not isinstance(name, str): From 6067fb30d711bf1991f6cbea6cb4ca831ce227d5 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Thu, 22 Dec 2022 06:08:07 +0100 Subject: [PATCH 24/49] Update dataframe.num_chunks() method to use to_batches --- python/pyarrow/interchange/dataframe.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/python/pyarrow/interchange/dataframe.py b/python/pyarrow/interchange/dataframe.py index 4c0eabd6b93..66cb5447268 100644 --- a/python/pyarrow/interchange/dataframe.py +++ b/python/pyarrow/interchange/dataframe.py @@ -116,7 +116,12 @@ def num_chunks(self) -> int: """ Return the number of chunks the DataFrame consists of. """ - return self._df.column(0).num_chunks + # pyarrow.Table can have columns with different number + # of chunks so we take the number of chunks that + # .to_batches() returns as it takes the min chunk size + # of all the columns (to_batches is a zero copy method) + batches = self._df.to_batches() + return len(batches) def column_names(self) -> Iterable[str]: """ From c6eb5f3c0fad872858a4b41026cdd1457fb3cef1 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Thu, 22 Dec 2022 06:30:20 +0100 Subject: [PATCH 25/49] Check for sentinel values in the datetime more efficently --- python/pyarrow/interchange/from_dataframe.py | 28 +++++++++++++------- 1 file changed, 18 insertions(+), 10 deletions(-) diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index a3e24f9c6d6..c450d38af8f 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -334,6 +334,23 @@ def datetime_column_to_array( col.describe_null, col.offset) + null_kind, sentinel_val = col.describe_null + # If sentinel values are used to represent missing values + # then validity buffer and bytemask are None and we will + # substitute the mask buffer with the sentinel mask buffer + if null_kind == ColumnNullType.USE_SENTINEL: + import pyarrow.compute as pc + + int_arr = pa.Array.from_buffers(pa.int64(), + col.size(), + [None, data_pa_buffer], + offset=col.offset) + bytemask = pc.equal(int_arr, sentinel_val) + # Will need to invert once using the mask buffer + # to construct an array: + # sentinel_arr = pc.equal(int_arr, sentinel_val) + # bytemask = pc.invert(sentinel_arr) + # Constructing a pa.Array from data and validity buffer array = pa.Array.from_buffers( data_dtype, @@ -342,16 +359,7 @@ def datetime_column_to_array( offset=col.offset, ) - null_kind, sentinel_val = col.describe_null - if null_kind == ColumnNullType.USE_SENTINEL: - # pandas iNaT - if sentinel_val == -9223372036854775808: - bytemask = np.isnan(np.array([str(e) for e in array.to_pylist()], - dtype="datetime64[ns]")) - else: - raise NotImplementedError( - f"Sentinel value {sentinel_val} for datetime is not yet " - "supported.") + # In case a bytemask was constructed with validity_buffer() call # or with sentinel_value then we have to add the mask to the array From 1a67177789d1681757d3d810ba0e864a9aa80991 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Thu, 22 Dec 2022 15:08:19 +0100 Subject: [PATCH 26/49] Make bigger changes to how masks and arrays are constructed --- python/pyarrow/interchange/from_dataframe.py | 323 ++++++------------ .../pyarrow/tests/interchange/test_extra.py | 4 +- 2 files changed, 110 insertions(+), 217 deletions(-) diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index c450d38af8f..1f15711daab 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -27,11 +27,10 @@ ColumnNullType, ) -import ctypes -import numpy as np import pyarrow as pa import re +import pyarrow.compute as pc from pyarrow.interchange.column import Dtype @@ -54,7 +53,7 @@ DtypeKind.FLOAT: {16: pa.float16(), 32: pa.float32(), 64: pa.float64()}, - DtypeKind.BOOL: {1: pa.bool_()}, + DtypeKind.BOOL: {1: pa.uint8()}, DtypeKind.STRING: {8: pa.string()}, } @@ -105,7 +104,7 @@ def _from_dataframe(df: DataFrameObject, allow_copy=True): """ batches = [] for chunk in df.get_chunks(): - batch = protocol_df_chunk_to_pyarrow(chunk, allow_copy) + batch = protocol_df_chunk_to_pyarrow(chunk) batches.append(batch) table = pa.Table.from_batches(batches) @@ -114,7 +113,6 @@ def _from_dataframe(df: DataFrameObject, allow_copy=True): def protocol_df_chunk_to_pyarrow( df: DataFrameObject, - allow_copy: bool = True ) -> pa.Table: """ Convert interchange protocol chunk to ``pa.RecordBatch``. @@ -127,10 +125,8 @@ def protocol_df_chunk_to_pyarrow( ------- pa.RecordBatch """ - # We need a dict of columns here, with each column being a PyArrow - # or NumPy array. - columns: dict[str, pa.Array | np.ndarray] = {} - buffers = [] # hold on to buffers, keeps memory alive + # We need a dict of columns here, with each column being a pa.Array + columns: dict[str, pa.Array] = {} for name in df.column_names(): if not isinstance(name, str): raise ValueError(f"Column {name} is not a string") @@ -144,26 +140,22 @@ def protocol_df_chunk_to_pyarrow( DtypeKind.FLOAT, DtypeKind.STRING, ): - columns[name], buf = column_to_array(col, allow_copy) + columns[name] = column_to_array(col) elif dtype == DtypeKind.BOOL: - columns[name], buf = bool_8_column_to_array(col, allow_copy) + columns[name] = bool_8_column_to_array(col) elif dtype == DtypeKind.CATEGORICAL: - columns[name], buf = categorical_column_to_dictionary( - col, allow_copy) + columns[name] = categorical_column_to_dictionary(col) elif dtype == DtypeKind.DATETIME: - columns[name], buf = datetime_column_to_array(col, allow_copy) + columns[name] = datetime_column_to_array(col) else: raise NotImplementedError(f"Data type {dtype} not handled yet") - buffers.append(buf) - return pa.RecordBatch.from_pydict(columns) def column_to_array( col: ColumnObject, - allow_copy: bool = True -) -> tuple[pa.Array, Any]: +) -> pa.Array: """ Convert a column holding one of the primitive dtypes to a PyArrow array. A primitive type is one of: int, uint, float, bool (1 bit). @@ -174,23 +166,19 @@ def column_to_array( Returns ------- - tuple - Tuple of pa.Array holding the data and the memory owner object - that keeps the memory alive. + pa.Array """ buffers = col.get_buffers() null_count = col.null_count data = buffers_to_array(buffers, col.size(), col.describe_null, - col.dtype, col.offset, - allow_copy, null_count) - return data, buffers + col.offset, null_count) + return data def bool_8_column_to_array( col: ColumnObject, - allow_copy: bool = True -) -> tuple[pa.Array, Any]: +) -> pa.Array: """ Convert a column holding boolean dtype with bit width = 8 to a PyArrow array. @@ -201,57 +189,36 @@ def bool_8_column_to_array( Returns ------- - tuple - Tuple of pa.Array holding the data and the memory owner object - that keeps the memory alive. + pa.Array """ - if not allow_copy: - raise RuntimeError( - "PyArrow Array will always be created out of a NumPy ndarray " - "and a copy is required which is forbidden by allow_copy=False" - ) buffers = col.get_buffers() - # Data buffer - data_buff, data_dtype = buffers["data"] - offset = col.offset - data_bit_width = data_dtype[1] - - data = buffer_to_ndarray(data_buff, - data_bit_width, - offset) + data = buffers_to_array(buffers, col.size(), + col.describe_null, + col.offset) + data = pc.cast(data, pa.bool_()) # Validity buffer try: validity_buff, validity_dtype = buffers["validity"] except TypeError: validity_buff = None - - null_kind, sentinel_val = col.describe_null - bytemask = None + validity_pa_buff = validity_buff if validity_buff: - if null_kind in (ColumnNullType.USE_BYTEMASK, - ColumnNullType.USE_BITMASK): - - validity_bit_width = validity_dtype[1] - bytemask = buffer_to_ndarray(validity_buff, - validity_bit_width, - offset) - if sentinel_val == 0: - bytemask = np.invert(bytemask) - else: - raise NotImplementedError(f"{null_kind} null representation " - "is not yet supported for boolean " - "dtype.") - # Output - return pa.array(data, mask=bytemask), buffers + validity_pa_buff = validity_buffer(validity_buff, + validity_dtype, + col.describe_null, + col.offset) + + return pa.Array.from_buffers(pa.bool_(), col.size(), + [validity_pa_buff, data.buffers()[1]], + offset=col.offset) def categorical_column_to_dictionary( col: ColumnObject, - allow_copy: bool = True -) -> tuple[pa.DictionaryArray, Any]: +) -> pa.DictionaryArray: """ Convert a column holding categorical data to a pa.DictionaryArray. @@ -261,9 +228,7 @@ def categorical_column_to_dictionary( Returns ------- - tuple - Tuple of pa.DictionaryArray holding the data and the memory owner - object that keeps the memory alive. + pa.DictionaryArray """ categorical = col.describe_categorical null_kind, sentinel_val = col.describe_null @@ -273,33 +238,45 @@ def categorical_column_to_dictionary( "Non-dictionary categoricals not supported yet") cat_column = categorical["categories"] - dictionary = column_to_array(cat_column)[0] + dictionary = column_to_array(cat_column) buffers = col.get_buffers() - indices = buffers_to_array(buffers, col.size(), - col.describe_null, - col.dtype, col.offset, - allow_copy) - - if null_kind == ColumnNullType.USE_SENTINEL: - if not allow_copy: - raise RuntimeError( - "PyArrow Array will always be created out of a NumPy ndarray " - "and a copy is required which is forbidden by allow_copy=False" - ) - bytemask = [value == sentinel_val for value in indices.to_pylist()] - indices = pa.array(indices.to_pylist(), mask=bytemask) - dict_array = pa.DictionaryArray.from_arrays(indices, dictionary) - return dict_array, buffers + # Construct a pyarrow Buffer for the indices + data_buff, data_type = buffers["data"] + data_pa_buffer = pa.foreign_buffer(data_buff.ptr, data_buff.bufsize, + base=data_buff) + kind, bit_width, _, _ = data_type + indices_type = _PYARROW_DTYPES.get(kind, {}).get(bit_width, None) + + # Construct a pyarrow buffer for validity mask + try: + validity_buff, validity_dtype = buffers["validity"] + except TypeError: + validity_buff = None + + validity_pa_buff = validity_buff + if validity_buff: + validity_pa_buff = validity_buffer(validity_buff, + validity_dtype, + col.describe_null, + col.offset) + + # Constructing a pa.DictionaryArray from data and validity buffer + dtype = pa.dictionary(indices_type, dictionary.type, + categorical["is_ordered"]) + dict_array = pa.DictionaryArray.from_buffers(dtype, col.size(), + [validity_pa_buff, + data_pa_buffer], + dictionary, col.offset) + return dict_array def datetime_column_to_array( col: ColumnObject, - allow_copy: bool = True -) -> tuple[pa.Array, Any]: +) -> pa.Array: """ - Convert a column holding DateTime data to a NumPy array. + Convert a column holding DateTime data to a pa.Array. Parameters ---------- @@ -307,9 +284,7 @@ def datetime_column_to_array( Returns ------- - tuple - Tuple of pa.Array holding the data and the memory owner object - that keeps the memory alive. + pa.Array """ buffers = col.get_buffers() data_buff, data_type = buffers["data"] @@ -323,55 +298,21 @@ def datetime_column_to_array( data_dtype = pa.timestamp(unit, tz=tz) # Data buffer - data_pa_buffer = pa.foreign_buffer(data_buff.ptr, data_buff.bufsize) + data_pa_buffer = pa.foreign_buffer(data_buff.ptr, data_buff.bufsize, + base=data_buff) # Validity buffer validity_pa_buff = validity_buff - bytemask = None if validity_pa_buff: - validity_pa_buff, bytemask = validity_buffer(validity_buff, - validity_dtype, - col.describe_null, - col.offset) - - null_kind, sentinel_val = col.describe_null - # If sentinel values are used to represent missing values - # then validity buffer and bytemask are None and we will - # substitute the mask buffer with the sentinel mask buffer - if null_kind == ColumnNullType.USE_SENTINEL: - import pyarrow.compute as pc - - int_arr = pa.Array.from_buffers(pa.int64(), - col.size(), - [None, data_pa_buffer], - offset=col.offset) - bytemask = pc.equal(int_arr, sentinel_val) - # Will need to invert once using the mask buffer - # to construct an array: - # sentinel_arr = pc.equal(int_arr, sentinel_val) - # bytemask = pc.invert(sentinel_arr) + validity_pa_buff = validity_buffer(validity_buff, + validity_dtype, + col.describe_null, + col.offset) # Constructing a pa.Array from data and validity buffer - array = pa.Array.from_buffers( - data_dtype, - col.size(), - [validity_pa_buff, data_pa_buffer], - offset=col.offset, - ) - - - - # In case a bytemask was constructed with validity_buffer() call - # or with sentinel_value then we have to add the mask to the array - if bytemask is not None: - if not allow_copy: - raise RuntimeError( - "PyArrow Array will always be created out of a NumPy ndarray " - "and a copy is required which is forbidden by allow_copy=False" - ) - return pa.array(array.to_pylist(), mask=bytemask), buffers - else: - return array, buffers + return pa.Array.from_buffers(data_dtype, col.size(), + [validity_pa_buff, data_pa_buffer], + offset=col.offset) def parse_datetime_format_str(format_str): @@ -396,9 +337,7 @@ def buffers_to_array( buffers: ColumnBuffers, length: int, describe_null: ColumnNullType, - validity_dtype: Dtype, offset: int = 0, - allow_copy: bool = True, null_count: int = -1 ) -> pa.Array: """ @@ -414,14 +353,8 @@ def buffers_to_array( describe_null: ColumnNullType Null representation the column dtype uses, as a tuple ``(kind, value)`` - validity_dtype : Dtype, - Dtype description as a tuple ``(kind, bit-width, format string, - endianness)``. offset : int, default: 0 Number of elements to offset from the start of the buffer. - allow_copy : bool, default: True - Whether to allow copying the memory to perform the conversion - (if false then zero-copy approach is requested). null_count : int, default: -1 Returns @@ -444,7 +377,7 @@ def buffers_to_array( except TypeError: offset_buff = None - kind, bit_width, _, _ = data_type + kind, bit_width, f_string, _ = data_type data_dtype = _PYARROW_DTYPES.get(kind, {}).get(bit_width, None) if data_dtype is None: @@ -452,24 +385,25 @@ def buffers_to_array( f"Conversion for {data_type} is not yet supported.") # Construct a pyarrow Buffer - data_pa_buffer = pa.foreign_buffer(data_buff.ptr, data_buff.bufsize) + data_pa_buffer = pa.foreign_buffer(data_buff.ptr, data_buff.bufsize, + base=data_buff) # Construct a validity pyarrow Buffer or a bytemask, if exists validity_pa_buff = validity_buff - _, _, validity_f_string, _ = validity_dtype - bytemask = None if validity_pa_buff: - validity_pa_buff, bytemask = validity_buffer(validity_buff, - validity_dtype, - describe_null, - offset) + validity_pa_buff = validity_buffer(validity_buff, + validity_dtype, + describe_null, + offset) # Construct a pyarrow Array from buffers if offset_buff: # If an offset buffer exists, construct an offset pyarrow Buffer # and add it to the construction of an array - offset_pa_buffer = pa.py_buffer(offset_buff._x) - if validity_f_string == 'U': + offset_pa_buffer = pa.foreign_buffer(offset_buff.ptr, + offset_buff.bufsize, + base=offset_buff) + if f_string == 'U': if not null_count: null_count = -1 array = pa.LargeStringArray.from_buffers( @@ -495,27 +429,18 @@ def buffers_to_array( offset=offset, ) - # In case a bytemask was constructed with validity_buffer() call - # then we have to add the mask to the array - if bytemask is not None: - if not allow_copy: - raise RuntimeError( - "PyArrow Array will always be created out of a NumPy ndarray " - "and a copy is required which is forbidden by allow_copy=False" - ) - return pa.array(array.to_pylist(), mask=bytemask) - else: - return array + return array def validity_buffer( validity_buff: BufferObject, validity_dtype: Dtype, describe_null: ColumnNullType, + length: int, offset: int = 0, -) -> tuple[pa.Buffer, np.ndarray]: +) -> pa.Buffer: """ - Build a PyArrow buffer or a NumPy array from the passed buffer. + Build a PyArrow buffer from the passed buffer. Parameters ---------- @@ -527,72 +452,40 @@ def validity_buffer( describe_null : ColumnNullType Null representation the column dtype uses, as a tuple ``(kind, value)`` + length : int + The number of values in the array. offset : int, default: 0 Number of elements to offset from the start of the buffer. Returns ------- - tuple - Tuple of a pyarrow buffer and a np.ndarray defining a mask. - Ony one of both can be not None. + pa.Buffer """ null_kind, sentinel_val = describe_null - validity_kind, validity_bit_width, _, _ = validity_dtype + validity_kind, _, _, _ = validity_dtype assert validity_kind == DtypeKind.BOOL - bytemask = None - if null_kind == ColumnNullType.USE_BYTEMASK: - - bytemask = buffer_to_ndarray(validity_buff, - validity_bit_width, - offset) - if sentinel_val == 0: - bytemask = np.invert(bytemask) - validity_pa_buff = None + if null_kind == ColumnNullType.USE_BYTEMASK or ( + null_kind == ColumnNullType.USE_BITMASK and sentinel_val == 1 + ): + + validity_pa_buff = pa.foreign_buffer(validity_buff.ptr, + validity_buff.bufsize, + base=validity_buff) + mask = pa.Array.from_buffers(pa.uint8(), length, + [None, validity_pa_buff], + offset=offset) + if sentinel_val == 1: + mask = pc.invert(mask) + mask_bool = pc.cast(mask, pa.bool_()) + validity_buff = mask_bool.buffers()[1] elif null_kind == ColumnNullType.USE_BITMASK and sentinel_val == 0: - validity_pa_buff = pa.foreign_buffer( - validity_buff.ptr, validity_buff.bufsize) + validity_buff = pa.foreign_buffer(validity_buff.ptr, + validity_buff.bufsize, + base=validity_buff) else: raise NotImplementedError( f"{describe_null} null representation is not yet supported.") - return validity_pa_buff, bytemask - - -def buffer_to_ndarray( - validity_buff: BufferObject, - bit_width: int, - offset: int = 0, -) -> np.ndarray: - """ - Build a NumPy array from the passed buffer. - - Parameters - ---------- - validity_buff : BufferObject - Tuple of underlying validity buffer and associated dtype. - bit_width : int - The number of bits of the buffer dtype as an integer. - offset : int, default: 0 - Number of elements to offset from the start of the buffer. - - Returns - ------- - np.ndarray - """ - # TODO: No DLPack yet, so need to construct a new ndarray from - # the data pointer and size in the buffer plus the dtype on the - # column. Use DLPack as NumPy supports it since - # https://github.com/numpy/numpy/pull/19083 - ctypes_type = np.ctypeslib.as_ctypes_type(bool) - data_pointer = ctypes.cast( - validity_buff.ptr + (offset * bit_width // 8), - ctypes.POINTER(ctypes_type) - ) - - data = np.ctypeslib.as_array( - data_pointer, shape=(validity_buff.bufsize // (bit_width // 8),) - ) - - return data + return validity_buff diff --git a/python/pyarrow/tests/interchange/test_extra.py b/python/pyarrow/tests/interchange/test_extra.py index 954fe28a3be..720a6a1045b 100644 --- a/python/pyarrow/tests/interchange/test_extra.py +++ b/python/pyarrow/tests/interchange/test_extra.py @@ -224,7 +224,7 @@ def test_roundtrip_pandas_boolean(): @pytest.mark.pandas @pytest.mark.parametrize("unit", ['s', 'ms', 'us', 'ns']) def test_roundtrip_pandas_datetime(unit): - # pandas always creates datetime64 in "us" + # pandas always creates datetime64 in "ns" # resolution, timezones are not yet supported if Version(pd.__version__) < Version("1.5.0"): @@ -233,7 +233,7 @@ def test_roundtrip_pandas_datetime(unit): dt_arr = [dt(2007, 7, 13), dt(2007, 7, 14), dt(2007, 7, 15)] table = pa.table({"a": pa.array(dt_arr, type=pa.timestamp(unit))}) - expected = pa.table({"a": pa.array(dt_arr, type=pa.timestamp('us'))}) + expected = pa.table({"a": pa.array(dt_arr, type=pa.timestamp('ns'))}) from pandas.core.interchange.from_dataframe import ( from_dataframe as pandas_from_dataframe From 51dcc49395016da2080bf2ae13a5757751e83a85 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Thu, 22 Dec 2022 15:16:28 +0100 Subject: [PATCH 27/49] Import from pandas.api.interchange --- python/pyarrow/tests/interchange/test_extra.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/python/pyarrow/tests/interchange/test_extra.py b/python/pyarrow/tests/interchange/test_extra.py index 720a6a1045b..b24033340ac 100644 --- a/python/pyarrow/tests/interchange/test_extra.py +++ b/python/pyarrow/tests/interchange/test_extra.py @@ -177,7 +177,7 @@ def test_pandas_roundtrip(uint, int, float, np_float): } ) - from pandas.core.interchange.from_dataframe import ( + from pandas.api.interchange import ( from_dataframe as pandas_from_dataframe ) pandas_df = pandas_from_dataframe(table) @@ -204,7 +204,7 @@ def test_roundtrip_pandas_boolean(): table = pa.table({"a": [True, False, True]}) expected = pa.table({"a": pa.array([1, 0, 1], type=pa.uint8())}) - from pandas.core.interchange.from_dataframe import ( + from pandas.api.interchange import ( from_dataframe as pandas_from_dataframe ) pandas_df = pandas_from_dataframe(table) @@ -235,7 +235,7 @@ def test_roundtrip_pandas_datetime(unit): table = pa.table({"a": pa.array(dt_arr, type=pa.timestamp(unit))}) expected = pa.table({"a": pa.array(dt_arr, type=pa.timestamp('ns'))}) - from pandas.core.interchange.from_dataframe import ( + from pandas.api.interchange import ( from_dataframe as pandas_from_dataframe ) pandas_df = pandas_from_dataframe(table) @@ -263,7 +263,7 @@ def test_pandas_assertion_error_large_string(): arr = pa.array(data, type=pa.large_string()) table = pa.table([arr], names=["large_string"]) - from pandas.core.interchange.from_dataframe import ( + from pandas.api.interchange import ( from_dataframe as pandas_from_dataframe ) From 4879ef271b9cda2a5da2ba47e5d1f74c6f5aa587 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Thu, 22 Dec 2022 15:49:29 +0100 Subject: [PATCH 28/49] Add a check for use_nan, correct test using np.nan and put back check for sentinel values deleted by accident --- python/pyarrow/interchange/from_dataframe.py | 37 ++++++++++++++++--- .../pyarrow/tests/interchange/test_extra.py | 7 ++-- 2 files changed, 36 insertions(+), 8 deletions(-) diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index 1f15711daab..8cc5f41de46 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -256,7 +256,14 @@ def categorical_column_to_dictionary( validity_buff = None validity_pa_buff = validity_buff - if validity_buff: + if null_kind == ColumnNullType.USE_SENTINEL: + indices = buffers_to_array(buffers, col.size(), + col.describe_null, + col.offset) + sentinel_arr = pc.equal(indices, sentinel_val) + mask_bool = pc.invert(sentinel_arr) + validity_pa_buff = mask_bool.buffers()[1] + elif validity_buff: validity_pa_buff = validity_buffer(validity_buff, validity_dtype, col.describe_null, @@ -303,7 +310,16 @@ def datetime_column_to_array( # Validity buffer validity_pa_buff = validity_buff - if validity_pa_buff: + null_kind, sentinel_val = col.describe_null + if null_kind == ColumnNullType.USE_SENTINEL: + int_arr = pa.Array.from_buffers(pa.int64(), + col.size(), + [None, data_pa_buffer], + offset=col.offset) + sentinel_arr = pc.equal(int_arr, sentinel_val) + mask_bool = pc.invert(sentinel_arr) + validity_pa_buff = mask_bool.buffers()[1] + elif validity_buff: validity_pa_buff = validity_buffer(validity_buff, validity_dtype, col.describe_null, @@ -388,6 +404,18 @@ def buffers_to_array( data_pa_buffer = pa.foreign_buffer(data_buff.ptr, data_buff.bufsize, base=data_buff) + # Check for float NaN values + null_kind, _ = describe_null + if null_kind == ColumnNullType.USE_NAN: + pyarrow_data = pa.Array.from_buffers( + data_dtype, + length, + [None, data_pa_buffer], + offset=offset, + ) + mask = pc.is_nan(pyarrow_data) + mask = pc.invert(mask) + validity_pa_buff = mask.buffers()[1] # Construct a validity pyarrow Buffer or a bytemask, if exists validity_pa_buff = validity_buff if validity_pa_buff: @@ -468,16 +496,15 @@ def validity_buffer( if null_kind == ColumnNullType.USE_BYTEMASK or ( null_kind == ColumnNullType.USE_BITMASK and sentinel_val == 1 ): - validity_pa_buff = pa.foreign_buffer(validity_buff.ptr, validity_buff.bufsize, base=validity_buff) mask = pa.Array.from_buffers(pa.uint8(), length, [None, validity_pa_buff], offset=offset) - if sentinel_val == 1: - mask = pc.invert(mask) mask_bool = pc.cast(mask, pa.bool_()) + if sentinel_val == 1: + mask_bool = pc.invert(mask_bool) validity_buff = mask_bool.buffers()[1] elif null_kind == ColumnNullType.USE_BITMASK and sentinel_val == 0: diff --git a/python/pyarrow/tests/interchange/test_extra.py b/python/pyarrow/tests/interchange/test_extra.py index b24033340ac..8051ff523f7 100644 --- a/python/pyarrow/tests/interchange/test_extra.py +++ b/python/pyarrow/tests/interchange/test_extra.py @@ -280,8 +280,8 @@ def test_pandas_assertion_error_large_string(): @pytest.mark.parametrize( "float, np_float", [ (pa.float16(), np.float16), - # (pa.float32(), np.float32), #errors due to inequality in nan - # (pa.float64(), np.float64) #errors due to inequality in nan + (pa.float32(), np.float32), + (pa.float64(), np.float64) ] ) @pytest.mark.parametrize("unit", ['s', 'ms', 'us', 'ns']) @@ -296,7 +296,8 @@ def test_pyarrow_roundtrip(uint, int, float, np_float, unit, tz): { "a": pa.array(arr, type=uint), "b": pa.array(arr, type=int), - "c": pa.array(np.array(arr, dtype=np_float), type=float), + "c": pa.array(np.array(arr, dtype=np_float), + type=float, from_pandas=True), "d": [True, False, True], "e": ["a", "", "c"], "f": pa.array(dt_arr, type=pa.timestamp(unit, tz=tz)) From 1cbd5949aa8f7a2338f474a921574639a6c38424 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Wed, 4 Jan 2023 08:25:13 +0100 Subject: [PATCH 29/49] Add test coverage for pandas -> pyarrow conversion --- python/pyarrow/interchange/from_dataframe.py | 106 +++++++++--------- .../pyarrow/tests/interchange/test_extra.py | 71 +++++++++++- 2 files changed, 122 insertions(+), 55 deletions(-) diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index 8cc5f41de46..4973c0ca2b1 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -209,6 +209,7 @@ def bool_8_column_to_array( validity_pa_buff = validity_buffer(validity_buff, validity_dtype, col.describe_null, + col.size(), col.offset) return pa.Array.from_buffers(pa.bool_(), col.size(), @@ -231,7 +232,6 @@ def categorical_column_to_dictionary( pa.DictionaryArray """ categorical = col.describe_categorical - null_kind, sentinel_val = col.describe_null if not categorical["is_dictionary"]: raise NotImplementedError( @@ -242,40 +242,13 @@ def categorical_column_to_dictionary( buffers = col.get_buffers() - # Construct a pyarrow Buffer for the indices - data_buff, data_type = buffers["data"] - data_pa_buffer = pa.foreign_buffer(data_buff.ptr, data_buff.bufsize, - base=data_buff) - kind, bit_width, _, _ = data_type - indices_type = _PYARROW_DTYPES.get(kind, {}).get(bit_width, None) + indices = buffers_to_array(buffers, col.size(), + col.describe_null, + col.offset) - # Construct a pyarrow buffer for validity mask - try: - validity_buff, validity_dtype = buffers["validity"] - except TypeError: - validity_buff = None + # Constructing a pa.DictionaryArray + dict_array = pa.DictionaryArray.from_arrays(indices, dictionary) - validity_pa_buff = validity_buff - if null_kind == ColumnNullType.USE_SENTINEL: - indices = buffers_to_array(buffers, col.size(), - col.describe_null, - col.offset) - sentinel_arr = pc.equal(indices, sentinel_val) - mask_bool = pc.invert(sentinel_arr) - validity_pa_buff = mask_bool.buffers()[1] - elif validity_buff: - validity_pa_buff = validity_buffer(validity_buff, - validity_dtype, - col.describe_null, - col.offset) - - # Constructing a pa.DictionaryArray from data and validity buffer - dtype = pa.dictionary(indices_type, dictionary.type, - categorical["is_ordered"]) - dict_array = pa.DictionaryArray.from_buffers(dtype, col.size(), - [validity_pa_buff, - data_pa_buffer], - dictionary, col.offset) return dict_array @@ -323,6 +296,7 @@ def datetime_column_to_array( validity_pa_buff = validity_buffer(validity_buff, validity_dtype, col.describe_null, + col.size(), col.offset) # Constructing a pa.Array from data and validity buffer @@ -389,7 +363,7 @@ def buffers_to_array( except TypeError: validity_buff = None try: - offset_buff, _ = buffers["offsets"] + offset_buff, offset_dtype = buffers["offsets"] except TypeError: offset_buff = None @@ -404,33 +378,50 @@ def buffers_to_array( data_pa_buffer = pa.foreign_buffer(data_buff.ptr, data_buff.bufsize, base=data_buff) - # Check for float NaN values - null_kind, _ = describe_null - if null_kind == ColumnNullType.USE_NAN: - pyarrow_data = pa.Array.from_buffers( - data_dtype, - length, - [None, data_pa_buffer], - offset=offset, - ) - mask = pc.is_nan(pyarrow_data) - mask = pc.invert(mask) - validity_pa_buff = mask.buffers()[1] # Construct a validity pyarrow Buffer or a bytemask, if exists validity_pa_buff = validity_buff - if validity_pa_buff: + null_kind, sentinel_val = describe_null + + # Check for float NaN values + if null_kind == ColumnNullType.USE_NAN: + if kind == DtypeKind.FLOAT and bit_width == 16: + raise NotImplementedError( + f"{data_type} with {null_kind} is not yet supported.") + else: + pyarrow_data = pa.Array.from_buffers( + data_dtype, + length, + [None, data_pa_buffer], + offset=offset, + ) + mask = pc.is_nan(pyarrow_data) + mask = pc.invert(mask) + validity_pa_buff = mask.buffers()[1] + # Check for sentinel values values + elif null_kind == ColumnNullType.USE_SENTINEL: + int_arr = pa.Array.from_buffers(data_dtype, + length, + [None, data_pa_buffer], + offset=offset) + sentinel_arr = pc.equal(int_arr, sentinel_val) + mask_bool = pc.invert(sentinel_arr) + validity_pa_buff = mask_bool.buffers()[1] + elif validity_buff: validity_pa_buff = validity_buffer(validity_buff, validity_dtype, describe_null, + length, offset) # Construct a pyarrow Array from buffers if offset_buff: + _, offset_bit_width, _, _ = offset_dtype # If an offset buffer exists, construct an offset pyarrow Buffer # and add it to the construction of an array offset_pa_buffer = pa.foreign_buffer(offset_buff.ptr, offset_buff.bufsize, base=offset_buff) + if f_string == 'U': if not null_count: null_count = -1 @@ -443,8 +434,12 @@ def buffers_to_array( offset=offset, ) else: + if offset_bit_width == 64: + string_type = pa.large_string() + else: + string_type = pa.string() array = pa.Array.from_buffers( - pa.string(), + string_type, length, [validity_pa_buff, offset_pa_buffer, data_pa_buffer], offset=offset, @@ -499,10 +494,17 @@ def validity_buffer( validity_pa_buff = pa.foreign_buffer(validity_buff.ptr, validity_buff.bufsize, base=validity_buff) - mask = pa.Array.from_buffers(pa.uint8(), length, - [None, validity_pa_buff], - offset=offset) - mask_bool = pc.cast(mask, pa.bool_()) + + if null_kind == ColumnNullType.USE_BYTEMASK: + mask = pa.Array.from_buffers(pa.int8(), length, + [None, validity_pa_buff], + offset=offset) + mask_bool = pc.cast(mask, pa.bool_()) + else: + mask_bool = pa.Array.from_buffers(pa.bool_(), length, + [None, validity_pa_buff], + offset=offset) + if sentinel_val == 1: mask_bool = pc.invert(mask_bool) validity_buff = mask_bool.buffers()[1] diff --git a/python/pyarrow/tests/interchange/test_extra.py b/python/pyarrow/tests/interchange/test_extra.py index 8051ff523f7..c9c71d3efd9 100644 --- a/python/pyarrow/tests/interchange/test_extra.py +++ b/python/pyarrow/tests/interchange/test_extra.py @@ -224,8 +224,8 @@ def test_roundtrip_pandas_boolean(): @pytest.mark.pandas @pytest.mark.parametrize("unit", ['s', 'ms', 'us', 'ns']) def test_roundtrip_pandas_datetime(unit): - # pandas always creates datetime64 in "ns" - # resolution, timezones are not yet supported + # pandas < 2.0 always creates datetime64 in "ns" + # resolution, timezones are not yet supported in pandas if Version(pd.__version__) < Version("1.5.0"): pytest.skip("__dataframe__ added to pandas in 1.5.0") @@ -271,6 +271,71 @@ def test_pandas_assertion_error_large_string(): pandas_from_dataframe(table) +@pytest.mark.pandas +@pytest.mark.parametrize( + "np_float", [np.float32, np.float64] +) +def test_pandas_to_pyarrow_with_missing(np_float): + np_array = np.array([0, np.nan, 2], dtype=np_float) + datetime_array = [None, dt(2007, 7, 14), dt(2007, 7, 15)] + df = pd.DataFrame({ + "a": np_array, # float, ColumnNullType.USE_NAN + "dt": datetime_array # ColumnNullType.USE_SENTINEL + }) + expected = pa.table({ + "a": pa.array(np_array, from_pandas=True), + "dt": pa.array(datetime_array, type=pa.timestamp("ns")) + }) + result = pi.from_dataframe(df) + + assert result.equals(expected) + + +@pytest.mark.pandas +def test_pandas_to_pyarrow_float16_with_missing(): + # np.float16 errors if ps.is_nan is used + # pyarrow.lib.ArrowNotImplementedError: Function 'is_nan' has no kernel + # matching input types (halffloat) + np_array = np.array([0, np.nan, 2], dtype=np.float16) + df = pd.DataFrame({"a": np_array}) + + with pytest.raises(NotImplementedError): + pi.from_dataframe(df) + + +@pytest.mark.pandas +def test_pandas_to_pyarrow_string_with_missing(): + # pandas is using int64 offsets for string dtype so the constructed + # pyarrow string column will always be a large_string data type + arr = { + "Y": ["a", "b", None], # bool, ColumnNullType.USE_BYTEMASK, + } + df = pd.DataFrame(arr) + expected = pa.table(arr) + result = pi.from_dataframe(df) + + assert result[0].to_pylist() == expected[0].to_pylist() + assert pa.types.is_string(expected[0].type) + assert pa.types.is_large_string(result[0].type) + + +@pytest.mark.pandas +def test_pandas_to_pyarrow_categorical_with_missing(): + arr = ["Mon", "Tue", "Mon", "Wed", "Mon", "Thu", "Fri", "Sat", None] + df = pd.DataFrame( + {"weekday": arr} + ) + df = df.astype("category") + result = pi.from_dataframe(df) + + expected_dictionary = ["Fri", "Mon", "Sat", "Thu", "Tue", "Wed"] + expected_indices = pa.array([1, 4, 1, 5, 1, 3, 0, 2, None], type=pa.int8()) + + assert result[0].to_pylist() == arr + assert result[0].chunk(0).dictionary.to_pylist() == expected_dictionary + assert result[0].chunk(0).indices.equals(expected_indices) + + @pytest.mark.parametrize( "uint", [pa.uint8(), pa.uint16(), pa.uint32()] ) @@ -317,7 +382,7 @@ def test_pyarrow_roundtrip(uint, int, float, np_float, unit, tz): def test_pyarrow_roundtrip_categorical(): - arr = ["Mon", "Tue", "Mon", "Wed", "Mon", "Thu", "Fri", "Sat", "Sun"] + arr = ["Mon", "Tue", "Mon", "Wed", "Mon", "Thu", "Fri", None, "Sun"] table = pa.table( {"weekday": pa.array(arr).dictionary_encode()} ) From a6b6e54ea9c0fdc61b5b5fc7615ee4df350f5775 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Wed, 4 Jan 2023 08:27:39 +0100 Subject: [PATCH 30/49] Rename test_extra.py to test_conversion.py --- .../tests/interchange/{test_extra.py => test_conversion.py} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename python/pyarrow/tests/interchange/{test_extra.py => test_conversion.py} (100%) diff --git a/python/pyarrow/tests/interchange/test_extra.py b/python/pyarrow/tests/interchange/test_conversion.py similarity index 100% rename from python/pyarrow/tests/interchange/test_extra.py rename to python/pyarrow/tests/interchange/test_conversion.py From 2e36185293e6964c24d84206b9db57d0938f6c0e Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Wed, 4 Jan 2023 09:59:55 +0100 Subject: [PATCH 31/49] Skip pandas -> pyarrow tests for older versions of pandas --- python/pyarrow/tests/interchange/test_conversion.py | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/python/pyarrow/tests/interchange/test_conversion.py b/python/pyarrow/tests/interchange/test_conversion.py index c9c71d3efd9..24ceab6ea1b 100644 --- a/python/pyarrow/tests/interchange/test_conversion.py +++ b/python/pyarrow/tests/interchange/test_conversion.py @@ -276,6 +276,9 @@ def test_pandas_assertion_error_large_string(): "np_float", [np.float32, np.float64] ) def test_pandas_to_pyarrow_with_missing(np_float): + if Version(pd.__version__) < Version("1.5.0"): + pytest.skip("__dataframe__ added to pandas in 1.5.0") + np_array = np.array([0, np.nan, 2], dtype=np_float) datetime_array = [None, dt(2007, 7, 14), dt(2007, 7, 15)] df = pd.DataFrame({ @@ -293,6 +296,9 @@ def test_pandas_to_pyarrow_with_missing(np_float): @pytest.mark.pandas def test_pandas_to_pyarrow_float16_with_missing(): + if Version(pd.__version__) < Version("1.5.0"): + pytest.skip("__dataframe__ added to pandas in 1.5.0") + # np.float16 errors if ps.is_nan is used # pyarrow.lib.ArrowNotImplementedError: Function 'is_nan' has no kernel # matching input types (halffloat) @@ -305,6 +311,9 @@ def test_pandas_to_pyarrow_float16_with_missing(): @pytest.mark.pandas def test_pandas_to_pyarrow_string_with_missing(): + if Version(pd.__version__) < Version("1.5.0"): + pytest.skip("__dataframe__ added to pandas in 1.5.0") + # pandas is using int64 offsets for string dtype so the constructed # pyarrow string column will always be a large_string data type arr = { @@ -321,6 +330,9 @@ def test_pandas_to_pyarrow_string_with_missing(): @pytest.mark.pandas def test_pandas_to_pyarrow_categorical_with_missing(): + if Version(pd.__version__) < Version("1.5.0"): + pytest.skip("__dataframe__ added to pandas in 1.5.0") + arr = ["Mon", "Tue", "Mon", "Wed", "Mon", "Thu", "Fri", "Sat", None] df = pd.DataFrame( {"weekday": arr} From 4ca948d5f5edb88e99b1dea59e681a30d33ca26f Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Wed, 4 Jan 2023 11:38:27 +0100 Subject: [PATCH 32/49] Add test coverage for sliced table in pyarrow roundtrip --- python/pyarrow/interchange/from_dataframe.py | 38 ++++++++----------- .../tests/interchange/test_conversion.py | 12 ++++-- 2 files changed, 23 insertions(+), 27 deletions(-) diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index 4973c0ca2b1..c97555470bf 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -53,7 +53,7 @@ DtypeKind.FLOAT: {16: pa.float16(), 32: pa.float32(), 64: pa.float64()}, - DtypeKind.BOOL: {1: pa.uint8()}, + DtypeKind.BOOL: {8: pa.uint8()}, DtypeKind.STRING: {8: pa.string()}, } @@ -142,7 +142,7 @@ def protocol_df_chunk_to_pyarrow( ): columns[name] = column_to_array(col) elif dtype == DtypeKind.BOOL: - columns[name] = bool_8_column_to_array(col) + columns[name] = bool_column_to_array(col) elif dtype == DtypeKind.CATEGORICAL: columns[name] = categorical_column_to_dictionary(col) elif dtype == DtypeKind.DATETIME: @@ -176,12 +176,11 @@ def column_to_array( return data -def bool_8_column_to_array( +def bool_column_to_array( col: ColumnObject, ) -> pa.Array: """ - Convert a column holding boolean dtype with bit width = 8 to a - PyArrow array. + Convert a column holding boolean dtype to a PyArrow array. Parameters ---------- @@ -192,29 +191,12 @@ def bool_8_column_to_array( pa.Array """ buffers = col.get_buffers() - data = buffers_to_array(buffers, col.size(), col.describe_null, col.offset) data = pc.cast(data, pa.bool_()) - # Validity buffer - try: - validity_buff, validity_dtype = buffers["validity"] - except TypeError: - validity_buff = None - validity_pa_buff = validity_buff - - if validity_buff: - validity_pa_buff = validity_buffer(validity_buff, - validity_dtype, - col.describe_null, - col.size(), - col.offset) - - return pa.Array.from_buffers(pa.bool_(), col.size(), - [validity_pa_buff, data.buffers()[1]], - offset=col.offset) + return data def categorical_column_to_dictionary( @@ -292,6 +274,11 @@ def datetime_column_to_array( sentinel_arr = pc.equal(int_arr, sentinel_val) mask_bool = pc.invert(sentinel_arr) validity_pa_buff = mask_bool.buffers()[1] + elif null_kind == ColumnNullType.NON_NULLABLE: + # sliced array can have a NON_NULLABLE ColumnNullType due + # to no missing values in that slice of an array though the bitmask + # exists and validity_buff must be set to None in this case + validity_buff = validity_pa_buff = None elif validity_buff: validity_pa_buff = validity_buffer(validity_buff, validity_dtype, @@ -406,6 +393,11 @@ def buffers_to_array( sentinel_arr = pc.equal(int_arr, sentinel_val) mask_bool = pc.invert(sentinel_arr) validity_pa_buff = mask_bool.buffers()[1] + elif null_kind == ColumnNullType.NON_NULLABLE: + # sliced array can have a NON_NULLABLE ColumnNullType due + # to no missing values in that slice of an array though the bitmask + # exists and validity_buff must be set to None in this case + validity_buff = validity_pa_buff = None elif validity_buff: validity_pa_buff = validity_buffer(validity_buff, validity_dtype, diff --git a/python/pyarrow/tests/interchange/test_conversion.py b/python/pyarrow/tests/interchange/test_conversion.py index 24ceab6ea1b..12fce91421c 100644 --- a/python/pyarrow/tests/interchange/test_conversion.py +++ b/python/pyarrow/tests/interchange/test_conversion.py @@ -363,7 +363,9 @@ def test_pandas_to_pyarrow_categorical_with_missing(): ) @pytest.mark.parametrize("unit", ['s', 'ms', 'us', 'ns']) @pytest.mark.parametrize("tz", ['America/New_York', '+07:30', '-04:30']) -def test_pyarrow_roundtrip(uint, int, float, np_float, unit, tz): +@pytest.mark.parametrize("offset, length", [(0, 3), (0, 2), (0, 2), (2, 1)]) +def test_pyarrow_roundtrip(uint, int, float, np_float, + unit, tz, offset, length): from datetime import datetime as dt arr = [1, 2, None] @@ -375,11 +377,13 @@ def test_pyarrow_roundtrip(uint, int, float, np_float, unit, tz): "b": pa.array(arr, type=int), "c": pa.array(np.array(arr, dtype=np_float), type=float, from_pandas=True), - "d": [True, False, True], - "e": ["a", "", "c"], - "f": pa.array(dt_arr, type=pa.timestamp(unit, tz=tz)) + # "d": [True, False, True], + # "e": [True, False, None], + "f": ["a", "", "c"], + "g": pa.array(dt_arr, type=pa.timestamp(unit, tz=tz)) } ) + table = table.slice(offset, length) result = _from_dataframe(table.__dataframe__()) assert table.equals(result) From 719ab882471604f61c2c8400f3a6b8f92edde3e8 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Thu, 5 Jan 2023 06:11:02 +0100 Subject: [PATCH 33/49] Correct the handling of bitpacked booleans --- python/pyarrow/interchange/column.py | 29 +++++++++++-------- .../tests/interchange/test_conversion.py | 22 ++++++-------- 2 files changed, 26 insertions(+), 25 deletions(-) diff --git a/python/pyarrow/interchange/column.py b/python/pyarrow/interchange/column.py index 31860594138..9f90d11eed6 100644 --- a/python/pyarrow/interchange/column.py +++ b/python/pyarrow/interchange/column.py @@ -216,15 +216,25 @@ def __init__( "is forbidden by allow_copy=False" ) column = column.combine_chunks() - self._col = column + self._allow_copy = allow_copy - dtype = self._col.type - try: - bit_width = dtype.bit_width - except ValueError: # in case of a variable-length strings - bit_width = 8 - self._dtype = self._dtype_from_arrowdtype(dtype, bit_width) + if pa.types.is_boolean(column.type): + if not allow_copy: + raise RuntimeError( + "Boolean column will be casted to uint8 and a copy " + "is required which is forbidden by allow_copy=False" + ) + self._dtype = self._dtype_from_arrowdtype(column.type, 8) + self._col = pc.cast(column, pa.uint8()) + else: + self._col = column + dtype = self._col.type + try: + bit_width = dtype.bit_width + except ValueError: # in case of a variable-length strings + bit_width = 8 + self._dtype = self._dtype_from_arrowdtype(dtype, bit_width) def size(self) -> int: """ @@ -460,11 +470,6 @@ def _get_data_buffer( array = self._col dtype = self.dtype - # In case of boolean arrays, cast to uint8 array - # as bit packed buffers are not supported - if pa.types.is_boolean(array.type): - array = pc.cast(array, pa.uint8()) - dtype = _PyArrowColumn(array).dtype # In case of dictionary arrays, use indices # to define a buffer, codes are transferred through # describe_categorical() diff --git a/python/pyarrow/tests/interchange/test_conversion.py b/python/pyarrow/tests/interchange/test_conversion.py index 12fce91421c..98ca5ae9dfa 100644 --- a/python/pyarrow/tests/interchange/test_conversion.py +++ b/python/pyarrow/tests/interchange/test_conversion.py @@ -195,14 +195,10 @@ def test_pandas_roundtrip(uint, int, float, np_float): @pytest.mark.pandas def test_roundtrip_pandas_boolean(): - # Boolean pyarrow Arrays are casted to uint8 as bit packed boolean - # is not yet supported by the protocol - if Version(pd.__version__) < Version("1.5.0"): pytest.skip("__dataframe__ added to pandas in 1.5.0") table = pa.table({"a": [True, False, True]}) - expected = pa.table({"a": pa.array([1, 0, 1], type=pa.uint8())}) from pandas.api.interchange import ( from_dataframe as pandas_from_dataframe @@ -210,15 +206,15 @@ def test_roundtrip_pandas_boolean(): pandas_df = pandas_from_dataframe(table) result = pi.from_dataframe(pandas_df) - assert expected.equals(result) + assert table.equals(result) - expected_protocol = expected.__dataframe__() + table_protocol = table.__dataframe__() result_protocol = result.__dataframe__() - assert expected_protocol.num_columns() == result_protocol.num_columns() - assert expected_protocol.num_rows() == result_protocol.num_rows() - assert expected_protocol.num_chunks() == result_protocol.num_chunks() - assert expected_protocol.column_names() == result_protocol.column_names() + assert table_protocol.num_columns() == result_protocol.num_columns() + assert table_protocol.num_rows() == result_protocol.num_rows() + assert table_protocol.num_chunks() == result_protocol.num_chunks() + assert table_protocol.column_names() == result_protocol.column_names() @pytest.mark.pandas @@ -377,9 +373,9 @@ def test_pyarrow_roundtrip(uint, int, float, np_float, "b": pa.array(arr, type=int), "c": pa.array(np.array(arr, dtype=np_float), type=float, from_pandas=True), - # "d": [True, False, True], - # "e": [True, False, None], - "f": ["a", "", "c"], + "d": [True, False, True], + "e": [True, False, None], + "f": ["a", None, "c"], "g": pa.array(dt_arr, type=pa.timestamp(unit, tz=tz)) } ) From 91ea33517ee46deba1d1d0343dd237dc4752c30d Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Thu, 5 Jan 2023 06:16:27 +0100 Subject: [PATCH 34/49] Small change in slicing parametrization --- python/pyarrow/tests/interchange/test_conversion.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/python/pyarrow/tests/interchange/test_conversion.py b/python/pyarrow/tests/interchange/test_conversion.py index 98ca5ae9dfa..6dee1f47fef 100644 --- a/python/pyarrow/tests/interchange/test_conversion.py +++ b/python/pyarrow/tests/interchange/test_conversion.py @@ -359,7 +359,7 @@ def test_pandas_to_pyarrow_categorical_with_missing(): ) @pytest.mark.parametrize("unit", ['s', 'ms', 'us', 'ns']) @pytest.mark.parametrize("tz", ['America/New_York', '+07:30', '-04:30']) -@pytest.mark.parametrize("offset, length", [(0, 3), (0, 2), (0, 2), (2, 1)]) +@pytest.mark.parametrize("offset, length", [(0, 3), (0, 2), (1, 2), (2, 1)]) def test_pyarrow_roundtrip(uint, int, float, np_float, unit, tz, offset, length): @@ -393,11 +393,13 @@ def test_pyarrow_roundtrip(uint, int, float, np_float, assert table_protocol.column_names() == result_protocol.column_names() -def test_pyarrow_roundtrip_categorical(): +@pytest.mark.parametrize("offset, length", [(0, 10), (0, 2), (7, 3), (2, 1)]) +def test_pyarrow_roundtrip_categorical(offset, length): arr = ["Mon", "Tue", "Mon", "Wed", "Mon", "Thu", "Fri", None, "Sun"] table = pa.table( {"weekday": pa.array(arr).dictionary_encode()} ) + table = table.slice(offset, length) result = _from_dataframe(table.__dataframe__()) assert table.equals(result) From c74eb452b4e05e86b10eeed68d6daa9b8cc82c88 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Thu, 5 Jan 2023 10:42:46 +0100 Subject: [PATCH 35/49] Add a RuntimeError for boolean and categorical columns in from_dataframe.py if allow_copy=False --- python/pyarrow/interchange/from_dataframe.py | 33 +++++++++++++++++--- 1 file changed, 29 insertions(+), 4 deletions(-) diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index c97555470bf..24099e459f9 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -104,7 +104,7 @@ def _from_dataframe(df: DataFrameObject, allow_copy=True): """ batches = [] for chunk in df.get_chunks(): - batch = protocol_df_chunk_to_pyarrow(chunk) + batch = protocol_df_chunk_to_pyarrow(chunk, allow_copy) batches.append(batch) table = pa.Table.from_batches(batches) @@ -113,6 +113,7 @@ def _from_dataframe(df: DataFrameObject, allow_copy=True): def protocol_df_chunk_to_pyarrow( df: DataFrameObject, + allow_copy: bool = True ) -> pa.Table: """ Convert interchange protocol chunk to ``pa.RecordBatch``. @@ -120,6 +121,11 @@ def protocol_df_chunk_to_pyarrow( Parameters ---------- df : DataFrameObject + Object supporting the interchange protocol, i.e. `__dataframe__` + method. + allow_copy : bool, default: True + Whether to allow copying the memory to perform the conversion + (if false then zero-copy approach is requested). Returns ------- @@ -142,9 +148,9 @@ def protocol_df_chunk_to_pyarrow( ): columns[name] = column_to_array(col) elif dtype == DtypeKind.BOOL: - columns[name] = bool_column_to_array(col) + columns[name] = bool_column_to_array(col, allow_copy) elif dtype == DtypeKind.CATEGORICAL: - columns[name] = categorical_column_to_dictionary(col) + columns[name] = categorical_column_to_dictionary(col, allow_copy) elif dtype == DtypeKind.DATETIME: columns[name] = datetime_column_to_array(col) else: @@ -178,6 +184,7 @@ def column_to_array( def bool_column_to_array( col: ColumnObject, + allow_copy: bool = True, ) -> pa.Array: """ Convert a column holding boolean dtype to a PyArrow array. @@ -185,11 +192,20 @@ def bool_column_to_array( Parameters ---------- col : ColumnObject + allow_copy : bool, default: True + Whether to allow copying the memory to perform the conversion + (if false then zero-copy approach is requested). Returns ------- pa.Array """ + if not allow_copy: + raise RuntimeError( + "Boolean column will be casted from uint8 and a copy " + "is required which is forbidden by allow_copy=False" + ) + buffers = col.get_buffers() data = buffers_to_array(buffers, col.size(), col.describe_null, @@ -201,6 +217,7 @@ def bool_column_to_array( def categorical_column_to_dictionary( col: ColumnObject, + allow_copy: bool = True, ) -> pa.DictionaryArray: """ Convert a column holding categorical data to a pa.DictionaryArray. @@ -208,11 +225,20 @@ def categorical_column_to_dictionary( Parameters ---------- col : ColumnObject + allow_copy : bool, default: True + Whether to allow copying the memory to perform the conversion + (if false then zero-copy approach is requested). Returns ------- pa.DictionaryArray """ + if not allow_copy: + raise RuntimeError( + "Boolean column will be casted from uint8 and a copy " + "is required which is forbidden by allow_copy=False" + ) + categorical = col.describe_categorical if not categorical["is_dictionary"]: @@ -223,7 +249,6 @@ def categorical_column_to_dictionary( dictionary = column_to_array(cat_column) buffers = col.get_buffers() - indices = buffers_to_array(buffers, col.size(), col.describe_null, col.offset) From c137337836eda7ae1f291057f67d3447472f26f3 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Thu, 5 Jan 2023 11:13:25 +0100 Subject: [PATCH 36/49] Optimize datetime handling in from_dataframe --- python/pyarrow/interchange/from_dataframe.py | 79 ++++---------------- 1 file changed, 15 insertions(+), 64 deletions(-) diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index 24099e459f9..9cd4f2876c9 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -145,14 +145,13 @@ def protocol_df_chunk_to_pyarrow( DtypeKind.UINT, DtypeKind.FLOAT, DtypeKind.STRING, + DtypeKind.DATETIME, ): columns[name] = column_to_array(col) elif dtype == DtypeKind.BOOL: columns[name] = bool_column_to_array(col, allow_copy) elif dtype == DtypeKind.CATEGORICAL: columns[name] = categorical_column_to_dictionary(col, allow_copy) - elif dtype == DtypeKind.DATETIME: - columns[name] = datetime_column_to_array(col) else: raise NotImplementedError(f"Data type {dtype} not handled yet") @@ -259,64 +258,6 @@ def categorical_column_to_dictionary( return dict_array -def datetime_column_to_array( - col: ColumnObject, -) -> pa.Array: - """ - Convert a column holding DateTime data to a pa.Array. - - Parameters - ---------- - col : ColumnObject - - Returns - ------- - pa.Array - """ - buffers = col.get_buffers() - data_buff, data_type = buffers["data"] - try: - validity_buff, validity_dtype = buffers["validity"] - except TypeError: - validity_buff = None - - format_str = data_type[2] - unit, tz = parse_datetime_format_str(format_str) - data_dtype = pa.timestamp(unit, tz=tz) - - # Data buffer - data_pa_buffer = pa.foreign_buffer(data_buff.ptr, data_buff.bufsize, - base=data_buff) - - # Validity buffer - validity_pa_buff = validity_buff - null_kind, sentinel_val = col.describe_null - if null_kind == ColumnNullType.USE_SENTINEL: - int_arr = pa.Array.from_buffers(pa.int64(), - col.size(), - [None, data_pa_buffer], - offset=col.offset) - sentinel_arr = pc.equal(int_arr, sentinel_val) - mask_bool = pc.invert(sentinel_arr) - validity_pa_buff = mask_bool.buffers()[1] - elif null_kind == ColumnNullType.NON_NULLABLE: - # sliced array can have a NON_NULLABLE ColumnNullType due - # to no missing values in that slice of an array though the bitmask - # exists and validity_buff must be set to None in this case - validity_buff = validity_pa_buff = None - elif validity_buff: - validity_pa_buff = validity_buffer(validity_buff, - validity_dtype, - col.describe_null, - col.size(), - col.offset) - - # Constructing a pa.Array from data and validity buffer - return pa.Array.from_buffers(data_dtype, col.size(), - [validity_pa_buff, data_pa_buffer], - offset=col.offset) - - def parse_datetime_format_str(format_str): """Parse datetime `format_str` to interpret the `data`.""" @@ -381,7 +322,13 @@ def buffers_to_array( kind, bit_width, f_string, _ = data_type - data_dtype = _PYARROW_DTYPES.get(kind, {}).get(bit_width, None) + # Map date type to pyarrow date type + if kind == DtypeKind.DATETIME: + unit, tz = parse_datetime_format_str(f_string) + data_dtype = pa.timestamp(unit, tz=tz) + else: + data_dtype = _PYARROW_DTYPES.get(kind, {}).get(bit_width, None) + # Error if dtype is not supported if data_dtype is None: raise NotImplementedError( f"Conversion for {data_type} is not yet supported.") @@ -409,9 +356,13 @@ def buffers_to_array( mask = pc.is_nan(pyarrow_data) mask = pc.invert(mask) validity_pa_buff = mask.buffers()[1] - # Check for sentinel values values + # Check for sentinel values elif null_kind == ColumnNullType.USE_SENTINEL: - int_arr = pa.Array.from_buffers(data_dtype, + if kind == DtypeKind.DATETIME: + sentinel_dtype = pa.int64() + else: + sentinel_dtype = data_dtype + int_arr = pa.Array.from_buffers(sentinel_dtype, length, [None, data_pa_buffer], offset=offset) @@ -419,7 +370,7 @@ def buffers_to_array( mask_bool = pc.invert(sentinel_arr) validity_pa_buff = mask_bool.buffers()[1] elif null_kind == ColumnNullType.NON_NULLABLE: - # sliced array can have a NON_NULLABLE ColumnNullType due + # Sliced array can have a NON_NULLABLE ColumnNullType due # to no missing values in that slice of an array though the bitmask # exists and validity_buff must be set to None in this case validity_buff = validity_pa_buff = None From 1e9cef9b21effc14c220d0dd07a72ed36679975e Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Thu, 5 Jan 2023 11:56:03 +0100 Subject: [PATCH 37/49] Optimize buffers_to_array in from_dataframe.py --- python/pyarrow/interchange/from_dataframe.py | 192 ++++++++++++------- 1 file changed, 122 insertions(+), 70 deletions(-) diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index 9cd4f2876c9..ed6dba30081 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -276,6 +276,24 @@ def parse_datetime_format_str(format_str): raise NotImplementedError(f"DateTime kind is not supported: {format_str}") +def map_date_type(data_type): + """Map column date type to pyarrow date type. """ + kind, bit_width, f_string, _ = data_type + + if kind == DtypeKind.DATETIME: + unit, tz = parse_datetime_format_str(f_string) + return pa.timestamp(unit, tz=tz) + else: + pa_dtype = _PYARROW_DTYPES.get(kind, {}).get(bit_width, None) + + # Error if dtype is not supported + if pa_dtype: + return pa_dtype + else: + raise NotImplementedError( + f"Conversion for {data_type} is not yet supported.") + + def buffers_to_array( buffers: ColumnBuffers, length: int, @@ -320,68 +338,27 @@ def buffers_to_array( except TypeError: offset_buff = None - kind, bit_width, f_string, _ = data_type - - # Map date type to pyarrow date type - if kind == DtypeKind.DATETIME: - unit, tz = parse_datetime_format_str(f_string) - data_dtype = pa.timestamp(unit, tz=tz) - else: - data_dtype = _PYARROW_DTYPES.get(kind, {}).get(bit_width, None) - # Error if dtype is not supported - if data_dtype is None: - raise NotImplementedError( - f"Conversion for {data_type} is not yet supported.") - # Construct a pyarrow Buffer data_pa_buffer = pa.foreign_buffer(data_buff.ptr, data_buff.bufsize, base=data_buff) - # Construct a validity pyarrow Buffer or a bytemask, if exists - validity_pa_buff = validity_buff - null_kind, sentinel_val = describe_null - - # Check for float NaN values - if null_kind == ColumnNullType.USE_NAN: - if kind == DtypeKind.FLOAT and bit_width == 16: - raise NotImplementedError( - f"{data_type} with {null_kind} is not yet supported.") - else: - pyarrow_data = pa.Array.from_buffers( - data_dtype, - length, - [None, data_pa_buffer], - offset=offset, - ) - mask = pc.is_nan(pyarrow_data) - mask = pc.invert(mask) - validity_pa_buff = mask.buffers()[1] - # Check for sentinel values - elif null_kind == ColumnNullType.USE_SENTINEL: - if kind == DtypeKind.DATETIME: - sentinel_dtype = pa.int64() - else: - sentinel_dtype = data_dtype - int_arr = pa.Array.from_buffers(sentinel_dtype, - length, - [None, data_pa_buffer], - offset=offset) - sentinel_arr = pc.equal(int_arr, sentinel_val) - mask_bool = pc.invert(sentinel_arr) - validity_pa_buff = mask_bool.buffers()[1] - elif null_kind == ColumnNullType.NON_NULLABLE: - # Sliced array can have a NON_NULLABLE ColumnNullType due - # to no missing values in that slice of an array though the bitmask - # exists and validity_buff must be set to None in this case - validity_buff = validity_pa_buff = None - elif validity_buff: - validity_pa_buff = validity_buffer(validity_buff, - validity_dtype, - describe_null, - length, - offset) + # Construct a validity pyarrow Buffer, if applicable + if validity_buff: + validity_pa_buff = validity_buffer_from_mask(validity_buff, + validity_dtype, + describe_null, + length, + offset) + else: + validity_pa_buff = validity_buffer_nan_sentinel(data_pa_buffer, + data_type, + describe_null, + length, + offset) # Construct a pyarrow Array from buffers + data_dtype = map_date_type(data_type) + if offset_buff: _, offset_bit_width, _, _ = offset_dtype # If an offset buffer exists, construct an offset pyarrow Buffer @@ -390,7 +367,7 @@ def buffers_to_array( offset_buff.bufsize, base=offset_buff) - if f_string == 'U': + if data_type[2] == 'U': if not null_count: null_count = -1 array = pa.LargeStringArray.from_buffers( @@ -423,7 +400,7 @@ def buffers_to_array( return array -def validity_buffer( +def validity_buffer_from_mask( validity_buff: BufferObject, validity_dtype: Dtype, describe_null: ColumnNullType, @@ -431,7 +408,7 @@ def validity_buffer( offset: int = 0, ) -> pa.Buffer: """ - Build a PyArrow buffer from the passed buffer. + Build a PyArrow buffer from the passed mask buffer. Parameters ---------- @@ -456,33 +433,108 @@ def validity_buffer( validity_kind, _, _, _ = validity_dtype assert validity_kind == DtypeKind.BOOL - if null_kind == ColumnNullType.USE_BYTEMASK or ( + if null_kind == ColumnNullType.NON_NULLABLE: + # Sliced array can have a NON_NULLABLE ColumnNullType due + # to no missing values in that slice of an array though the bitmask + # exists and validity_buff must be set to None in this case + return None + + elif null_kind == ColumnNullType.USE_BYTEMASK or ( null_kind == ColumnNullType.USE_BITMASK and sentinel_val == 1 ): - validity_pa_buff = pa.foreign_buffer(validity_buff.ptr, - validity_buff.bufsize, - base=validity_buff) + buff = pa.foreign_buffer(validity_buff.ptr, + validity_buff.bufsize, + base=validity_buff) if null_kind == ColumnNullType.USE_BYTEMASK: mask = pa.Array.from_buffers(pa.int8(), length, - [None, validity_pa_buff], + [None, buff], offset=offset) mask_bool = pc.cast(mask, pa.bool_()) else: mask_bool = pa.Array.from_buffers(pa.bool_(), length, - [None, validity_pa_buff], + [None, buff], offset=offset) if sentinel_val == 1: mask_bool = pc.invert(mask_bool) - validity_buff = mask_bool.buffers()[1] + + return mask_bool.buffers()[1] elif null_kind == ColumnNullType.USE_BITMASK and sentinel_val == 0: - validity_buff = pa.foreign_buffer(validity_buff.ptr, - validity_buff.bufsize, - base=validity_buff) + return pa.foreign_buffer(validity_buff.ptr, + validity_buff.bufsize, + base=validity_buff) else: raise NotImplementedError( f"{describe_null} null representation is not yet supported.") - return validity_buff + +def validity_buffer_nan_sentinel( + data_pa_buffer: BufferObject, + data_type: Dtype, + describe_null: ColumnNullType, + length: int, + offset: int = 0, +) -> pa.Buffer: + """ + Build a PyArrow buffer from NaN or sentinel values. + + Parameters + ---------- + data_pa_buffer : pa.Buffer + PyArrow buffer for the column data. + data_type : Dtype + Dtype description as a tuple ``(kind, bit-width, format string, + endianness)``. + describe_null : ColumnNullType + Null representation the column dtype uses, + as a tuple ``(kind, value)`` + length : int + The number of values in the array. + offset : int, default: 0 + Number of elements to offset from the start of the buffer. + + Returns + ------- + pa.Buffer + """ + kind, bit_width, _, _ = data_type + data_dtype = map_date_type(data_type) + null_kind, sentinel_val = describe_null + + # Check for float NaN values + if null_kind == ColumnNullType.USE_NAN: + if kind == DtypeKind.FLOAT and bit_width == 16: + raise NotImplementedError( + f"{data_type} with {null_kind} is not yet supported.") + else: + pyarrow_data = pa.Array.from_buffers( + data_dtype, + length, + [None, data_pa_buffer], + offset=offset, + ) + mask = pc.is_nan(pyarrow_data) + mask = pc.invert(mask) + return mask.buffers()[1] + + # Check for sentinel values + elif null_kind == ColumnNullType.USE_SENTINEL: + if kind == DtypeKind.DATETIME: + sentinel_dtype = pa.int64() + else: + sentinel_dtype = data_dtype + int_arr = pa.Array.from_buffers(sentinel_dtype, + length, + [None, data_pa_buffer], + offset=offset) + sentinel_arr = pc.equal(int_arr, sentinel_val) + mask_bool = pc.invert(sentinel_arr) + return mask_bool.buffers()[1] + + elif null_kind == ColumnNullType.NON_NULLABLE: + pass + else: + raise NotImplementedError( + f"{describe_null} null representation is not yet supported.") From 0c539a04612d6022f6bd91aa09b2d14fc43003e0 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Tue, 10 Jan 2023 10:05:01 +0100 Subject: [PATCH 38/49] Apply suggestions from code review - Joris Co-authored-by: Joris Van den Bossche --- python/pyarrow/interchange/column.py | 3 ++- python/pyarrow/interchange/from_dataframe.py | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/python/pyarrow/interchange/column.py b/python/pyarrow/interchange/column.py index 9f90d11eed6..6ebe036cbe3 100644 --- a/python/pyarrow/interchange/column.py +++ b/python/pyarrow/interchange/column.py @@ -232,7 +232,8 @@ def __init__( dtype = self._col.type try: bit_width = dtype.bit_width - except ValueError: # in case of a variable-length strings + except ValueError: + # in case of a variable-length strings, considered as array of bytes (8 bits) bit_width = 8 self._dtype = self._dtype_from_arrowdtype(dtype, bit_width) diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index ed6dba30081..3890c4f8d73 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -114,7 +114,7 @@ def _from_dataframe(df: DataFrameObject, allow_copy=True): def protocol_df_chunk_to_pyarrow( df: DataFrameObject, allow_copy: bool = True -) -> pa.Table: +) -> pa.RecordBatch: """ Convert interchange protocol chunk to ``pa.RecordBatch``. From 6399be386daba1aafa369292cbad0bfc38594ca6 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Tue, 10 Jan 2023 10:16:36 +0100 Subject: [PATCH 39/49] Add string column back to test_pandas_roundtrip for pandas versions 2.0 and higher --- python/pyarrow/tests/interchange/test_conversion.py | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/python/pyarrow/tests/interchange/test_conversion.py b/python/pyarrow/tests/interchange/test_conversion.py index 6dee1f47fef..83aff6cd0c5 100644 --- a/python/pyarrow/tests/interchange/test_conversion.py +++ b/python/pyarrow/tests/interchange/test_conversion.py @@ -169,13 +169,12 @@ def test_pandas_roundtrip(uint, int, float, np_float): "a": pa.array(arr, type=uint), "b": pa.array(arr, type=int), "c": pa.array(np.array(arr, dtype=np_float), type=float), - # String dtype errors as col.size is called as a property - # not method in pandas, see L255-L257 in - # pandas/core/interchange/from_dataframe.py - # "d": ["a", "", "c"], - # large string is not supported by pandas implementation } ) + if Version(pd.__version__) >= Version("2.0"): + # See https://github.com/pandas-dev/pandas/issues/50554 + table["d"] = ["a", "", "c"] + # large string is not supported by pandas implementation from pandas.api.interchange import ( from_dataframe as pandas_from_dataframe From 9f68fe71ad5b05f6a6cf0e77c2606ffa730a76dc Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Tue, 10 Jan 2023 10:21:16 +0100 Subject: [PATCH 40/49] Fix linter error --- python/pyarrow/interchange/column.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/pyarrow/interchange/column.py b/python/pyarrow/interchange/column.py index 6ebe036cbe3..a9b89586165 100644 --- a/python/pyarrow/interchange/column.py +++ b/python/pyarrow/interchange/column.py @@ -233,7 +233,8 @@ def __init__( try: bit_width = dtype.bit_width except ValueError: - # in case of a variable-length strings, considered as array of bytes (8 bits) + # in case of a variable-length strings, considered as array + # of bytes (8 bits) bit_width = 8 self._dtype = self._dtype_from_arrowdtype(dtype, bit_width) From b9260662f93a337d2673104f4d3aebc0d63ea8c6 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Tue, 10 Jan 2023 10:23:19 +0100 Subject: [PATCH 41/49] Remove pandas specific comment for nan_as_null in dataframe.py --- python/pyarrow/interchange/dataframe.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/python/pyarrow/interchange/dataframe.py b/python/pyarrow/interchange/dataframe.py index 66cb5447268..d0717e02e88 100644 --- a/python/pyarrow/interchange/dataframe.py +++ b/python/pyarrow/interchange/dataframe.py @@ -53,9 +53,7 @@ def __init__( self._df = df # ``nan_as_null`` is a keyword intended for the consumer to tell the # producer to overwrite null values in the data with ``NaN`` (or - # ``NaT``). This currently has no effect; once support for nullable - # extension dtypes is added, this value should be propagated to - # columns. + # ``NaT``). if nan_as_null is True: raise RuntimeError( "nan_as_null=True currently has no effect, " From 5c5d25ee3e19bcaea52e64004005d11fc8beaa89 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Tue, 10 Jan 2023 10:27:11 +0100 Subject: [PATCH 42/49] Fix typo boolen -> categorical in categorical_column_to_dictionary --- python/pyarrow/interchange/from_dataframe.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index 3890c4f8d73..9a8547ca54e 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -234,7 +234,7 @@ def categorical_column_to_dictionary( """ if not allow_copy: raise RuntimeError( - "Boolean column will be casted from uint8 and a copy " + "Categorical column will be casted from uint8 and a copy " "is required which is forbidden by allow_copy=False" ) From f2a65a67f58b19078640fc4eabc1cb3b2c09b697 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Tue, 10 Jan 2023 11:07:04 +0100 Subject: [PATCH 43/49] Add a comment for float16 NotImplementedError in validity_buffer_nan_sentinel --- python/pyarrow/interchange/from_dataframe.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index 9a8547ca54e..482028dd1d8 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -506,6 +506,8 @@ def validity_buffer_nan_sentinel( # Check for float NaN values if null_kind == ColumnNullType.USE_NAN: if kind == DtypeKind.FLOAT and bit_width == 16: + # 'pyarrow.compute.is_nan' kernel not yet implemented + # for float16 raise NotImplementedError( f"{data_type} with {null_kind} is not yet supported.") else: From 075e888ec5ab9b3de1d1ffb1d755ce01512315b0 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Tue, 10 Jan 2023 11:16:00 +0100 Subject: [PATCH 44/49] Update validity_buffer_nan_sentinel in python/pyarrow/interchange/from_dataframe.py Co-authored-by: Joris Van den Bossche --- python/pyarrow/interchange/from_dataframe.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index 482028dd1d8..2a82d46f28f 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -527,11 +527,11 @@ def validity_buffer_nan_sentinel( sentinel_dtype = pa.int64() else: sentinel_dtype = data_dtype - int_arr = pa.Array.from_buffers(sentinel_dtype, + pyarrow_data = pa.Array.from_buffers(sentinel_dtype, length, [None, data_pa_buffer], offset=offset) - sentinel_arr = pc.equal(int_arr, sentinel_val) + sentinel_arr = pc.equal(pyarrow_data, sentinel_val) mask_bool = pc.invert(sentinel_arr) return mask_bool.buffers()[1] From efa12d6c85885e221192b147c4a312a2e4961a81 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Tue, 10 Jan 2023 11:31:47 +0100 Subject: [PATCH 45/49] Make change to the offset buffers part of buffers_to_array --- python/pyarrow/interchange/from_dataframe.py | 28 ++++++-------------- 1 file changed, 8 insertions(+), 20 deletions(-) diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index 2a82d46f28f..8586b15a803 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -174,10 +174,9 @@ def column_to_array( pa.Array """ buffers = col.get_buffers() - null_count = col.null_count data = buffers_to_array(buffers, col.size(), col.describe_null, - col.offset, null_count) + col.offset) return data @@ -299,7 +298,6 @@ def buffers_to_array( length: int, describe_null: ColumnNullType, offset: int = 0, - null_count: int = -1 ) -> pa.Array: """ Build a PyArrow array from the passed buffer. @@ -316,7 +314,6 @@ def buffers_to_array( as a tuple ``(kind, value)`` offset : int, default: 0 Number of elements to offset from the start of the buffer. - null_count : int, default: -1 Returns ------- @@ -368,27 +365,18 @@ def buffers_to_array( base=offset_buff) if data_type[2] == 'U': - if not null_count: - null_count = -1 - array = pa.LargeStringArray.from_buffers( - length, - offset_pa_buffer, - data_pa_buffer, - validity_pa_buff, - null_count=null_count, - offset=offset, - ) + string_type = pa.large_string() else: if offset_bit_width == 64: string_type = pa.large_string() else: string_type = pa.string() - array = pa.Array.from_buffers( - string_type, - length, - [validity_pa_buff, offset_pa_buffer, data_pa_buffer], - offset=offset, - ) + array = pa.Array.from_buffers( + string_type, + length, + [validity_pa_buff, offset_pa_buffer, data_pa_buffer], + offset=offset, + ) else: array = pa.Array.from_buffers( data_dtype, From 858cadbe1091365a91ca570f83b135ad98d8797a Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Tue, 10 Jan 2023 11:42:38 +0100 Subject: [PATCH 46/49] Linter correction --- python/pyarrow/interchange/from_dataframe.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index 8586b15a803..b2f65508a3e 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -516,9 +516,9 @@ def validity_buffer_nan_sentinel( else: sentinel_dtype = data_dtype pyarrow_data = pa.Array.from_buffers(sentinel_dtype, - length, - [None, data_pa_buffer], - offset=offset) + length, + [None, data_pa_buffer], + offset=offset) sentinel_arr = pc.equal(pyarrow_data, sentinel_val) mask_bool = pc.invert(sentinel_arr) return mask_bool.buffers()[1] From e937b4c59d91473885e21119439d4e686204f402 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Tue, 10 Jan 2023 15:52:24 +0100 Subject: [PATCH 47/49] Update the handling of allow_copy keyword --- python/pyarrow/interchange/from_dataframe.py | 47 +++++++++++++++++-- .../tests/interchange/test_conversion.py | 46 ++++++++++++++++++ 2 files changed, 88 insertions(+), 5 deletions(-) diff --git a/python/pyarrow/interchange/from_dataframe.py b/python/pyarrow/interchange/from_dataframe.py index b2f65508a3e..204530a3354 100644 --- a/python/pyarrow/interchange/from_dataframe.py +++ b/python/pyarrow/interchange/from_dataframe.py @@ -82,7 +82,8 @@ def from_dataframe(df: DataFrameObject, allow_copy=True) -> pa.Table: if not hasattr(df, "__dataframe__"): raise ValueError("`df` does not support __dataframe__") - return _from_dataframe(df.__dataframe__(allow_copy=allow_copy)) + return _from_dataframe(df.__dataframe__(allow_copy=allow_copy), + allow_copy=allow_copy) def _from_dataframe(df: DataFrameObject, allow_copy=True): @@ -147,7 +148,7 @@ def protocol_df_chunk_to_pyarrow( DtypeKind.STRING, DtypeKind.DATETIME, ): - columns[name] = column_to_array(col) + columns[name] = column_to_array(col, allow_copy) elif dtype == DtypeKind.BOOL: columns[name] = bool_column_to_array(col, allow_copy) elif dtype == DtypeKind.CATEGORICAL: @@ -160,6 +161,7 @@ def protocol_df_chunk_to_pyarrow( def column_to_array( col: ColumnObject, + allow_copy: bool = True, ) -> pa.Array: """ Convert a column holding one of the primitive dtypes to a PyArrow array. @@ -168,6 +170,9 @@ def column_to_array( Parameters ---------- col : ColumnObject + allow_copy : bool, default: True + Whether to allow copying the memory to perform the conversion + (if false then zero-copy approach is requested). Returns ------- @@ -176,7 +181,8 @@ def column_to_array( buffers = col.get_buffers() data = buffers_to_array(buffers, col.size(), col.describe_null, - col.offset) + col.offset, + allow_copy) return data @@ -298,6 +304,7 @@ def buffers_to_array( length: int, describe_null: ColumnNullType, offset: int = 0, + allow_copy: bool = True, ) -> pa.Array: """ Build a PyArrow array from the passed buffer. @@ -314,6 +321,9 @@ def buffers_to_array( as a tuple ``(kind, value)`` offset : int, default: 0 Number of elements to offset from the start of the buffer. + allow_copy : bool, default: True + Whether to allow copying the memory to perform the conversion + (if false then zero-copy approach is requested). Returns ------- @@ -345,13 +355,15 @@ def buffers_to_array( validity_dtype, describe_null, length, - offset) + offset, + allow_copy) else: validity_pa_buff = validity_buffer_nan_sentinel(data_pa_buffer, data_type, describe_null, length, - offset) + offset, + allow_copy) # Construct a pyarrow Array from buffers data_dtype = map_date_type(data_type) @@ -394,6 +406,7 @@ def validity_buffer_from_mask( describe_null: ColumnNullType, length: int, offset: int = 0, + allow_copy: bool = True, ) -> pa.Buffer: """ Build a PyArrow buffer from the passed mask buffer. @@ -412,6 +425,9 @@ def validity_buffer_from_mask( The number of values in the array. offset : int, default: 0 Number of elements to offset from the start of the buffer. + allow_copy : bool, default: True + Whether to allow copying the memory to perform the conversion + (if false then zero-copy approach is requested). Returns ------- @@ -435,6 +451,11 @@ def validity_buffer_from_mask( base=validity_buff) if null_kind == ColumnNullType.USE_BYTEMASK: + if not allow_copy: + raise RuntimeError( + "To create a bitmask a copy of the data is " + "required which is forbidden by allow_copy=False" + ) mask = pa.Array.from_buffers(pa.int8(), length, [None, buff], offset=offset) @@ -464,6 +485,7 @@ def validity_buffer_nan_sentinel( describe_null: ColumnNullType, length: int, offset: int = 0, + allow_copy: bool = True, ) -> pa.Buffer: """ Build a PyArrow buffer from NaN or sentinel values. @@ -482,6 +504,9 @@ def validity_buffer_nan_sentinel( The number of values in the array. offset : int, default: 0 Number of elements to offset from the start of the buffer. + allow_copy : bool, default: True + Whether to allow copying the memory to perform the conversion + (if false then zero-copy approach is requested). Returns ------- @@ -493,6 +518,12 @@ def validity_buffer_nan_sentinel( # Check for float NaN values if null_kind == ColumnNullType.USE_NAN: + if not allow_copy: + raise RuntimeError( + "To create a bitmask a copy of the data is " + "required which is forbidden by allow_copy=False" + ) + if kind == DtypeKind.FLOAT and bit_width == 16: # 'pyarrow.compute.is_nan' kernel not yet implemented # for float16 @@ -511,6 +542,12 @@ def validity_buffer_nan_sentinel( # Check for sentinel values elif null_kind == ColumnNullType.USE_SENTINEL: + if not allow_copy: + raise RuntimeError( + "To create a bitmask a copy of the data is " + "required which is forbidden by allow_copy=False" + ) + if kind == DtypeKind.DATETIME: sentinel_dtype = pa.int64() else: diff --git a/python/pyarrow/tests/interchange/test_conversion.py b/python/pyarrow/tests/interchange/test_conversion.py index 83aff6cd0c5..e3d33e7b77d 100644 --- a/python/pyarrow/tests/interchange/test_conversion.py +++ b/python/pyarrow/tests/interchange/test_conversion.py @@ -448,3 +448,49 @@ def test_nan_as_null(): table = pa.table({"a": [1, 2, 3, 4]}) with pytest.raises(RuntimeError): table.__dataframe__(nan_as_null=True) + + +@pytest.mark.pandas +def test_allow_copy_false(): + if Version(pd.__version__) < Version("1.5.0"): + pytest.skip("__dataframe__ added to pandas in 1.5.0") + + # Test that an error is raised when a copy is needed + # to create a bitmask + + df = pd.DataFrame({"a": [0, 1.0, 2.0]}) + with pytest.raises(RuntimeError): + pi.from_dataframe(df, allow_copy=False) + + df = pd.DataFrame({ + "dt": [None, dt(2007, 7, 14), dt(2007, 7, 15)] + }) + with pytest.raises(RuntimeError): + pi.from_dataframe(df, allow_copy=False) + + +@pytest.mark.pandas +def test_allow_copy_false_bool_categorical(): + if Version(pd.__version__) < Version("1.5.0"): + pytest.skip("__dataframe__ added to pandas in 1.5.0") + + # Test that an error is raised for boolean + # and categorical dtype (copy is always made) + + df = pd.DataFrame({"a": [None, False, True]}) + with pytest.raises(RuntimeError): + pi.from_dataframe(df, allow_copy=False) + + df = pd.DataFrame({"a": [True, False, True]}) + with pytest.raises(RuntimeError): + pi.from_dataframe(df, allow_copy=False) + + df = pd.DataFrame({"weekday": ["a", "b", None]}) + df = df.astype("category") + with pytest.raises(RuntimeError): + pi.from_dataframe(df, allow_copy=False) + + df = pd.DataFrame({"weekday": ["a", "b", "c"]}) + df = df.astype("category") + with pytest.raises(RuntimeError): + pi.from_dataframe(df, allow_copy=False) From 1b5f24836331a9cb34191694e65e9f165fa7de97 Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Thu, 12 Jan 2023 13:32:40 +0100 Subject: [PATCH 48/49] Fix failing nightly test --- python/pyarrow/tests/interchange/test_conversion.py | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/python/pyarrow/tests/interchange/test_conversion.py b/python/pyarrow/tests/interchange/test_conversion.py index e3d33e7b77d..6565312118e 100644 --- a/python/pyarrow/tests/interchange/test_conversion.py +++ b/python/pyarrow/tests/interchange/test_conversion.py @@ -219,16 +219,21 @@ def test_roundtrip_pandas_boolean(): @pytest.mark.pandas @pytest.mark.parametrize("unit", ['s', 'ms', 'us', 'ns']) def test_roundtrip_pandas_datetime(unit): - # pandas < 2.0 always creates datetime64 in "ns" - # resolution, timezones are not yet supported in pandas - if Version(pd.__version__) < Version("1.5.0"): pytest.skip("__dataframe__ added to pandas in 1.5.0") from datetime import datetime as dt + # timezones not included as they are not yet supported in + # the pandas implementation dt_arr = [dt(2007, 7, 13), dt(2007, 7, 14), dt(2007, 7, 15)] table = pa.table({"a": pa.array(dt_arr, type=pa.timestamp(unit))}) - expected = pa.table({"a": pa.array(dt_arr, type=pa.timestamp('ns'))}) + + if Version(pd.__version__) >= Version("2.0"): + expected = table + else: + # pandas < 2.0 always creates datetime64 in "ns" + # resolution + expected = pa.table({"a": pa.array(dt_arr, type=pa.timestamp('ns'))}) from pandas.api.interchange import ( from_dataframe as pandas_from_dataframe From 91394440e9496c5ab5575facbbb1ae5a6a8eed7a Mon Sep 17 00:00:00 2001 From: Alenka Frim Date: Thu, 12 Jan 2023 16:07:04 +0100 Subject: [PATCH 49/49] Fix the fix for the failing test --- .../tests/interchange/test_conversion.py | 39 +++++++++++++++---- 1 file changed, 31 insertions(+), 8 deletions(-) diff --git a/python/pyarrow/tests/interchange/test_conversion.py b/python/pyarrow/tests/interchange/test_conversion.py index 6565312118e..0680d9c4ec1 100644 --- a/python/pyarrow/tests/interchange/test_conversion.py +++ b/python/pyarrow/tests/interchange/test_conversion.py @@ -171,11 +171,6 @@ def test_pandas_roundtrip(uint, int, float, np_float): "c": pa.array(np.array(arr, dtype=np_float), type=float), } ) - if Version(pd.__version__) >= Version("2.0"): - # See https://github.com/pandas-dev/pandas/issues/50554 - table["d"] = ["a", "", "c"] - # large string is not supported by pandas implementation - from pandas.api.interchange import ( from_dataframe as pandas_from_dataframe ) @@ -192,6 +187,34 @@ def test_pandas_roundtrip(uint, int, float, np_float): assert table_protocol.column_names() == result_protocol.column_names() +@pytest.mark.pandas +def test_roundtrip_pandas_string(): + # See https://github.com/pandas-dev/pandas/issues/50554 + if Version(pd.__version__) < Version("1.6"): + pytest.skip(" Column.size() called as a method in pandas 2.0.0") + + # large string is not supported by pandas implementation + table = pa.table({"a": pa.array(["a", "", "c"])}) + + from pandas.api.interchange import ( + from_dataframe as pandas_from_dataframe + ) + pandas_df = pandas_from_dataframe(table) + result = pi.from_dataframe(pandas_df) + + assert result[0].to_pylist() == table[0].to_pylist() + assert pa.types.is_string(table[0].type) + assert pa.types.is_large_string(result[0].type) + + table_protocol = table.__dataframe__() + result_protocol = result.__dataframe__() + + assert table_protocol.num_columns() == result_protocol.num_columns() + assert table_protocol.num_rows() == result_protocol.num_rows() + assert table_protocol.num_chunks() == result_protocol.num_chunks() + assert table_protocol.column_names() == result_protocol.column_names() + + @pytest.mark.pandas def test_roundtrip_pandas_boolean(): if Version(pd.__version__) < Version("1.5.0"): @@ -228,12 +251,12 @@ def test_roundtrip_pandas_datetime(unit): dt_arr = [dt(2007, 7, 13), dt(2007, 7, 14), dt(2007, 7, 15)] table = pa.table({"a": pa.array(dt_arr, type=pa.timestamp(unit))}) - if Version(pd.__version__) >= Version("2.0"): - expected = table - else: + if Version(pd.__version__) < Version("1.6"): # pandas < 2.0 always creates datetime64 in "ns" # resolution expected = pa.table({"a": pa.array(dt_arr, type=pa.timestamp('ns'))}) + else: + expected = table from pandas.api.interchange import ( from_dataframe as pandas_from_dataframe