Learn more  » Push, build, and install  RubyGems npm packages Python packages Maven artifacts PHP packages Go Modules Bower components Debian packages RPM packages NuGet packages

arrow-nightlies / nanoarrow   python

Repository URL to install this package:

Version: 0.7.0.dev132 

/ src / nanoarrow / c_array.py

# 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 typing import Any, Iterable, Literal, Tuple, Union

from nanoarrow._array import CArray, CArrayBuilder, CArrayView, CDeviceArray
from nanoarrow._buffer import CBuffer, CBufferBuilder, NoneAwareWrapperIterator
from nanoarrow._device import DEVICE_CPU, Device
from nanoarrow._schema import CSchema, CSchemaBuilder
from nanoarrow._utils import obj_is_buffer, obj_is_capsule
from nanoarrow.c_buffer import c_buffer
from nanoarrow.c_schema import c_schema, c_schema_view

from nanoarrow import _types


def c_array(obj, schema=None) -> CArray:
    """ArrowArray wrapper

    This class provides a user-facing interface to access the fields of an ArrowArray
    as defined in the Arrow C Data interface, holding an optional reference to a
    :class:`CSchema` that can be used to safely deserialize the content.

    These objects are created using :func:`c_array`, which accepts any array-like
    object according to the Arrow PyCapsule interface, Python buffer protocol,
    or iterable of Python objects.

    This Python wrapper allows access to array fields but does not automatically
    deserialize their content: use :func:`c_array_view` to validate and deserialize
    the content into a more easily inspectable object.

    Note that the :class:`CArray` objects returned by ``.child()`` hold strong
    references to the original ``ArrowArray`` to avoid copies while inspecting an
    imported structure.

    Parameters
    ----------
    obj : array-like
        An object supporting the Arrow PyCapsule interface, the Python buffer
        protocol, or an iterable of Python objects.
    schema : schema-like or None
        A schema-like object as sanitized by :func:`c_schema` or None. This value
        will be used to request a data type from ``obj``; however, the conversion
        is best-effort (i.e., the data type of the returned ``CArray`` may be
        different than ``schema``).

    Examples
    --------

    >>> import nanoarrow as na
    >>> # Create from iterable
    >>> array = na.c_array([1, 2, 3], na.int32())
    >>> # Create from Python buffer (e.g., numpy array)
    >>> import numpy as np
    >>> array = na.c_array(np.array([1, 2, 3]))
    >>> # Create from Arrow PyCapsule (e.g., pyarrow array)
    >>> import pyarrow as pa
    >>> array = na.c_array(pa.array([1, 2, 3]))
    >>> # Access array fields
    >>> array.length
    3
    >>> array.null_count
    0
    """

    if schema is not None:
        schema = c_schema(schema)

    if isinstance(obj, CArray) and schema is None:
        return obj

    # Try Arrow PyCapsule protocol
    if hasattr(obj, "__arrow_c_array__"):
        schema_capsule = None if schema is None else schema.__arrow_c_schema__()
        return CArray._import_from_c_capsule(
            *obj.__arrow_c_array__(requested_schema=schema_capsule)
        )

    # Try import of bare capsule
    if obj_is_capsule(obj, "arrow_array"):
        if schema is None:
            schema_capsule = CSchema.allocate()._capsule
        else:
            schema_capsule = schema.__arrow_c_schema__()

        return CArray._import_from_c_capsule(schema_capsule, obj)

    # Try _export_to_c for Array/RecordBatch objects if pyarrow < 14.0
    if _obj_is_pyarrow_array(obj):
        out = CArray.allocate(CSchema.allocate())
        obj._export_to_c(out._addr(), out.schema._addr())
        return out

    # Use the ArrayBuilder classes to handle various strategies for other
    # types of objects (e.g., iterable, pybuffer, empty).
    try:
        builder_cls = _resolve_builder(obj)
    except Exception as e:
        raise TypeError(
            f"Can't convert object of type {type(obj).__name__} "
            f"to nanoarrow.c_array: \n {e}"
        ) from e

    try:
        if schema is None:
            obj, schema = builder_cls.infer_schema(obj)

        builder = builder_cls(schema)
        return builder.build_c_array(obj)
    except Exception as e:
        raise ValueError(
            f"An error occurred whilst converting {type(obj).__name__} "
            f"to nanoarrow.c_array: \n {e}"
        ) from e


def _resolve_builder(obj):
    if _obj_is_empty(obj):
        return EmptyArrayBuilder

    if obj_is_buffer(obj):
        return ArrayFromPyBufferBuilder

    if _obj_is_iterable(obj):
        return ArrayFromIterableBuilder

    raise TypeError(
        f"Can't resolve ArrayBuilder for object of type {type(obj).__name__}"
    )


def allocate_c_array(schema=None) -> CArray:
    """Allocate an uninitialized ArrowArray

    Examples
    --------

    >>> import pyarrow as pa
    >>> from nanoarrow.c_array import allocate_c_array
    >>> array = allocate_c_array()
    >>> pa.array([1, 2, 3])._export_to_c(array._addr())
    """
    if schema is not None:
        schema = c_schema(schema)

    return CArray.allocate(CSchema.allocate() if schema is None else schema)


def c_array_view(obj, schema=None) -> CArrayView:
    """ArrowArrayView wrapper

    The ``ArrowArrayView`` is a nanoarrow C library structure that provides
    structured access to buffers addresses, buffer sizes, and buffer
    data types. The buffer data is usually propagated from an ArrowArray
    but can also be propagated from other types of objects (e.g., serialized
    IPC). The offset and length of this view are independent of its parent
    (i.e., this object can also represent a slice of its parent).

    Examples
    --------

    >>> import pyarrow as pa
    >>> import numpy as np
    >>> import nanoarrow as na
    >>> from nanoarrow.c_array import c_array_view
    >>>
    >>> array = na.c_array(pa.array(["one", "two", "three", None]))
    >>> array_view = c_array_view(array)
    >>> np.array(array_view.buffer(1))
    array([ 0,  3,  6, 11, 11], dtype=int32)
    >>> np.array(array_view.buffer(2))
    array([b'o', b'n', b'e', b't', b'w', b'o', b't', b'h', b'r', b'e', b'e'],
          dtype='|S1')
    """

    if isinstance(obj, CArrayView) and schema is None:
        return obj

    return c_array(obj, schema).view()


def c_array_from_buffers(
    schema,
    length: int,
    buffers: Iterable[Any],
    null_count: int = -1,
    offset: int = 0,
    children: Iterable[Any] = (),
    validation_level: Literal[None, "full", "default", "minimal", "none"] = None,
    move: bool = False,
    device: Union[Device, None] = None,
) -> CArray:
    """Create an ArrowArray wrapper from components

    Given a schema, build an ArrowArray buffer-wise. This allows almost any array
    to be assembled; however, requires some knowledge of the Arrow Columnar
    specification. This function will do its best to validate the sizes and
    content of buffers according to ``validation_level``; however, not all
    types of arrays can currently be validated when constructed in this way.

    Parameters
    ----------
    schema : schema-like
        The data type of the desired array as sanitized by :func:`c_schema`.
    length : int
        The length of the output array.
    buffers : Iterable of buffer-like or None
        An iterable of buffers as sanitized by :func:`c_buffer`. Any object
        supporting the Python Buffer protocol is accepted. Buffer data types
        are not checked. A buffer value of ``None`` will skip setting a buffer
        (i.e., that buffer will be of length zero and its pointer will
        be ``NULL``).
    null_count : int, optional
        The number of null values, if known in advance. If -1 (the default),
        the null count will be calculated based on the validity bitmap. If
        the validity bitmap was set to ``None``, the calculated null count
        will be zero.
    offset : int, optional
        The logical offset from the start of the array.
    children : Iterable of array-like
        An iterable of arrays used to set child fields of the array. Can contain
        any object accepted by :func:`c_array`. Must contain the exact number of
        required children as specifed by ``schema``.
    validation_level: None or str, optional
        One of "none" (no check), "minimal" (check buffer sizes that do not require
        dereferencing buffer content), "default" (check all buffer sizes), or "full"
        (check all buffer sizes and all buffer content). The default, ``None``,
        will validate at the "default" level where possible.
    move : bool, optional
        Use ``True`` to move ownership of any input buffers or children to the
        output array.
    device : Device, optional
        An explicit device to use when constructing this array. If specified,
        this function will construct a :class:`CDeviceArray`; if unspecified,
        this function will construct a :class:`CArray` on the CPU device.

    Examples
    --------

    >>> import nanoarrow as na
    >>> c_array = na.c_array_from_buffers(na.uint8(), 5, [None, b"12345"])
    >>> na.Array(c_array).inspect()
    <ArrowArray uint8>
    - length: 5
    - offset: 0
    - null_count: 0
    - buffers[2]:
      - validity <bool[0 b] >
      - data <uint8[5 b] 49 50 51 52 53>
    - dictionary: NULL
    - children[0]:
    """
    if device is None:
        explicit_device = False
        device = DEVICE_CPU
    else:
        explicit_device = True

    schema = c_schema(schema)
    builder = CArrayBuilder.allocate(device)

    # Ensures that the output array->n_buffers is set and that the correct number
    # of children have been initialized.
    builder.init_from_schema(schema)

    # Set buffers, optionally moving ownership of the buffers as well (i.e.,
    # the objects in the input buffers would be replaced with an empty ArrowBuffer)
    for i, buffer in enumerate(buffers):
        if buffer is None:
            continue

        # If we're setting a CBuffer from something else, we can avoid an extra
        # level of Python wrapping by using move=True
        move = move or not isinstance(buffer, CBuffer)
        builder.set_buffer(i, c_buffer(buffer), move=move)

    # Set children, optionally moving ownership of the children as well (i.e.,
    # the objects in the input children would be marked released).
    n_children = 0
    for child_src in children:
        # If we're setting a CArray from something else, we can avoid an extra
        # level of Python wrapping by using move=True
        move = move or not isinstance(child_src, (CArray, CDeviceArray))
        if move and isinstance(child_src, CDeviceArray):
            child_src = child_src.array

        builder.set_child(n_children, c_array(child_src), move=move)
        n_children += 1

    if n_children != schema.n_children:
        raise ValueError(f"Expected {schema.n_children} children but got {n_children}")

    # Set array fields
    builder.set_length(length)
    builder.set_offset(offset)
    builder.set_null_count(null_count)

    # Calculates the null count if -1 (and if applicable)
    builder.resolve_null_count()

    # Validate + finish. If device is specified (even CPU), always
    # return a device array.
    if not explicit_device:
        return builder.finish(validation_level=validation_level)
    else:
        return builder.finish_device()


class ArrayBuilder:
    """Internal utility to build CArrays from various types of input

    This class and its subclasses are designed to help separate the code
    that actually builds a CArray from the code that chooses the strategy
    used to do the building.
    """

    @classmethod
    def infer_schema(cls, obj) -> Tuple[CSchema, Any]:
        """Infer the Arrow data type from a target object

        Returns the type as a :class:`CSchema` and an object that can be
        consumed in the same way by append() in the event it had to be
        modified to infer its type (e.g., for an iterable, it would be
        necessary to consume the first element from the original iterator).
        """
        raise NotImplementedError()

    def __init__(self, schema):
        self._schema = c_schema(schema)
        self._schema_view = c_schema_view(self._schema)
        self._c_builder = CArrayBuilder.allocate()
Loading ...