pyarrow.ChunkedArray — Apache Arrow v22.0.0 (original) (raw)
class pyarrow.ChunkedArray#
Bases: _PandasConvertible
An array-like composed from a (possibly empty) collection of pyarrow.Arrays
Warning
Do not call this class’s constructor directly.
Examples
To construct a ChunkedArray object use pyarrow.chunked_array():
import pyarrow as pa pa.chunked_array([], type=pa.int8()) <pyarrow.lib.ChunkedArray object at ...> [ ... ]
pa.chunked_array([[2, 2, 4], [4, 5, 100]]) <pyarrow.lib.ChunkedArray object at ...> [ [ 2, 2, 4 ], [ 4, 5, 100 ] ] isinstance(pa.chunked_array([[2, 2, 4], [4, 5, 100]]), pa.ChunkedArray) True
__init__(*args, **kwargs)#
Methods
Attributes
cast(self, target_type=None, safe=None, options=None)#
Cast array values to another data type
See pyarrow.compute.cast() for usage.
Parameters:
Type to cast array to.
Whether to check for conversion errors such as overflow.
optionsCastOptions, default None
Additional checks pass by CastOptions
Returns:
castArray or ChunkedArray
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, 5, 100]]) n_legs.type DataType(int64)
Change the data type of an array:
n_legs_seconds = n_legs.cast(pa.duration('s')) n_legs_seconds.type DurationType(duration[s])
chunk(self, i)#
Select a chunk by its index.
Parameters:
iint
Returns:
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, None], [4, 5, 100]]) n_legs.chunk(1) <pyarrow.lib.Int64Array object at ...> [ 4, 5, 100 ]
chunks#
Convert to a list of single-chunked arrays.
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, None], [4, 5, 100]]) n_legs <pyarrow.lib.ChunkedArray object at ...> [ [ 2, 2, null ], [ 4, 5, 100 ] ] n_legs.chunks [<pyarrow.lib.Int64Array object at ...> [ 2, 2, null ], <pyarrow.lib.Int64Array object at ...> [ 4, 5, 100 ]]
combine_chunks(self, MemoryPool memory_pool=None)#
Flatten this ChunkedArray into a single non-chunked array.
Parameters:
memory_poolMemoryPool, default None
For memory allocations, if required, otherwise use default pool
Returns:
resultArray
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, 5, 100]]) n_legs <pyarrow.lib.ChunkedArray object at ...> [ [ 2, 2, 4 ], [ 4, 5, 100 ] ] n_legs.combine_chunks() <pyarrow.lib.Int64Array object at ...> [ 2, 2, 4, 4, 5, 100 ]
data#
dictionary_encode(self, null_encoding='mask')#
Compute dictionary-encoded representation of array.
See pyarrow.compute.dictionary_encode() for full usage.
Parameters:
null_encodingstr, default “mask”
How to handle null entries.
Returns:
encodedChunkedArray
A dictionary-encoded version of this array.
Examples
import pyarrow as pa animals = pa.chunked_array(( ... ["Flamingo", "Parrot", "Dog"], ... ["Horse", "Brittle stars", "Centipede"] ... )) animals.dictionary_encode() <pyarrow.lib.ChunkedArray object at ...> [ ... -- dictionary: [ "Flamingo", "Parrot", "Dog", "Horse", "Brittle stars", "Centipede" ] -- indices: [ 0, 1, 2 ], ... -- dictionary: [ "Flamingo", "Parrot", "Dog", "Horse", "Brittle stars", "Centipede" ] -- indices: [ 3, 4, 5 ] ]
drop_null(self)#
Remove missing values from a chunked array. See pyarrow.compute.drop_null() for full description.
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, None], [4, 5, 100]]) n_legs <pyarrow.lib.ChunkedArray object at ...> [ [ 2, 2, null ], [ 4, 5, 100 ] ] n_legs.drop_null() <pyarrow.lib.ChunkedArray object at ...> [ [ 2, 2 ], [ 4, 5, 100 ] ]
equals(self, ChunkedArray other)#
Return whether the contents of two chunked arrays are equal.
Parameters:
otherpyarrow.ChunkedArray
Chunked array to compare against.
Returns:
are_equalbool
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, 5, 100]]) animals = pa.chunked_array(( ... ["Flamingo", "Parrot", "Dog"], ... ["Horse", "Brittle stars", "Centipede"] ... )) n_legs.equals(n_legs) True n_legs.equals(animals) False
fill_null(self, fill_value)#
Replace each null element in values with fill_value.
See pyarrow.compute.fill_null() for full usage.
Parameters:
fill_valueany
The replacement value for null entries.
Returns:
resultArray or ChunkedArray
A new array with nulls replaced by the given value.
Examples
import pyarrow as pa fill_value = pa.scalar(5, type=pa.int8()) n_legs = pa.chunked_array([[2, 2, 4], [4, None, 100]]) n_legs.fill_null(fill_value) <pyarrow.lib.ChunkedArray object at ...> [ [ 2, 2, 4, 4, 5, 100 ] ]
filter(self, mask, null_selection_behavior='drop')#
Select values from the chunked array.
See pyarrow.compute.filter() for full usage.
Parameters:
maskArray or array-like
The boolean mask to filter the chunked array with.
null_selection_behaviorstr, default “drop”
How nulls in the mask should be handled.
Returns:
filteredArray or ChunkedArray
An array of the same type, with only the elements selected by the boolean mask.
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, 5, 100]]) n_legs <pyarrow.lib.ChunkedArray object at ...> [ [ 2, 2, 4 ], [ 4, 5, 100 ] ] mask = pa.array([True, False, None, True, False, True]) n_legs.filter(mask) <pyarrow.lib.ChunkedArray object at ...> [ [ 2 ], [ 4, 100 ] ] n_legs.filter(mask, null_selection_behavior="emit_null") <pyarrow.lib.ChunkedArray object at ...> [ [ 2, null ], [ 4, 100 ] ]
flatten(self, MemoryPool memory_pool=None)#
Flatten this ChunkedArray. If it has a struct type, the column is flattened into one array per struct field.
Parameters:
memory_poolMemoryPool, default None
For memory allocations, if required, otherwise use default pool
Returns:
resultlist of ChunkedArray
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, 5, 100]]) c_arr = pa.chunked_array(n_legs.value_counts()) c_arr <pyarrow.lib.ChunkedArray object at ...> [ -- is_valid: all not null -- child 0 type: int64 [ 2, 4, 5, 100 ] -- child 1 type: int64 [ 2, 2, 1, 1 ] ] c_arr.flatten() [<pyarrow.lib.ChunkedArray object at ...> [ [ 2, 4, 5, 100 ] ], <pyarrow.lib.ChunkedArray object at ...> [ [ 2, 2, 1, 1 ] ]] c_arr.type StructType(struct<values: int64, counts: int64>) n_legs.type DataType(int64)
format(self, **kwargs)#
DEPRECATED, use pyarrow.ChunkedArray.to_string
Parameters:
**kwargsdict
Returns:
get_total_buffer_size(self)#
The sum of bytes in each buffer referenced by the chunked array.
An array may only reference a portion of a buffer. This method will overestimate in this case and return the byte size of the entire buffer.
If a buffer is referenced multiple times then it will only be counted once.
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, None, 100]]) n_legs.get_total_buffer_size() 49
index(self, value, start=None, end=None, *, memory_pool=None)#
Find the first index of a value.
See pyarrow.compute.index() for full usage.
Parameters:
valueScalar or object
The value to look for in the array.
startint, optional
The start index where to look for value.
endint, optional
The end index where to look for value.
memory_poolMemoryPool, optional
A memory pool for potential memory allocations.
Returns:
indexInt64Scalar
The index of the value in the array (-1 if not found).
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, 5, 100]]) n_legs <pyarrow.lib.ChunkedArray object at ...> [ [ 2, 2, 4 ], [ 4, 5, 100 ] ] n_legs.index(4) <pyarrow.Int64Scalar: 2> n_legs.index(4, start=3) <pyarrow.Int64Scalar: 3>
is_cpu#
Whether all chunks in the ChunkedArray are CPU-accessible.
is_nan(self)#
Return boolean array indicating the NaN values.
Examples
import pyarrow as pa import numpy as np arr = pa.chunked_array([[2, np.nan, 4], [4, None, 100]]) arr.is_nan() <pyarrow.lib.ChunkedArray object at ...> [ [ false, true, false, false, null, false ] ]
is_null(self, *, nan_is_null=False)#
Return boolean array indicating the null values.
Parameters:
nan_is_nullbool (optional, default False)
Whether floating-point NaN values should also be considered null.
Returns:
arraybool Array or ChunkedArray
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, None, 100]]) n_legs.is_null() <pyarrow.lib.ChunkedArray object at ...> [ [ false, false, false, false, true, false ] ]
is_valid(self)#
Return boolean array indicating the non-null values.
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, None, 100]]) n_legs.is_valid() <pyarrow.lib.ChunkedArray object at ...> [ [ true, true, true ], [ true, false, true ] ]
iterchunks(self)#
Convert to an iterator of ChunkArrays.
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, None, 100]]) for i in n_legs.iterchunks(): ... print(i.null_count) ... 0 1
length(self)#
Return length of a ChunkedArray.
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, 5, 100]]) n_legs.length() 6
nbytes#
Total number of bytes consumed by the elements of the chunked array.
In other words, the sum of bytes from all buffer ranges referenced.
Unlike get_total_buffer_size this method will account for array offsets.
If buffers are shared between arrays then the shared portion will only be counted multiple times.
The dictionary of dictionary arrays will always be counted in their entirety even if the array only references a portion of the dictionary.
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, None, 100]]) n_legs.nbytes 49
null_count#
Number of null entries
Returns:
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, None, 100]]) n_legs.null_count 1
num_chunks#
Number of underlying chunks.
Returns:
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, None], [4, 5, 100]]) n_legs.num_chunks 2
slice(self, offset=0, length=None)#
Compute zero-copy slice of this ChunkedArray
Parameters:
offsetint, default 0
Offset from start of array to slice
Length of slice (default is until end of batch starting from offset)
Returns:
slicedChunkedArray
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, 5, 100]]) n_legs <pyarrow.lib.ChunkedArray object at ...> [ [ 2, 2, 4 ], [ 4, 5, 100 ] ] n_legs.slice(2,2) <pyarrow.lib.ChunkedArray object at ...> [ [ 4 ], [ 4 ] ]
sort(self, order='ascending', **kwargs)#
Sort the ChunkedArray
Parameters:
orderstr, default “ascending”
Which order to sort values in. Accepted values are “ascending”, “descending”.
**kwargsdict, optional
Additional sorting options. As allowed by SortOptions
Returns:
resultChunkedArray
take(self, indices)#
Select values from the chunked array.
See pyarrow.compute.take() for full usage.
Parameters:
indicesArray or array-like
The indices in the array whose values will be returned.
Returns:
takenArray or ChunkedArray
An array with the same datatype, containing the taken values.
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, 5, 100]]) n_legs <pyarrow.lib.ChunkedArray object at ...> [ [ 2, 2, 4 ], [ 4, 5, 100 ] ] n_legs.take([1,4,5]) <pyarrow.lib.ChunkedArray object at ...> [ [ 2, 5, 100 ] ]
to_numpy(self, zero_copy_only=False)#
Return a NumPy copy of this array (experimental).
Parameters:
zero_copy_onlybool, default False
Introduced for signature consistence with pyarrow.Array.to_numpy. This must be False here since NumPy arrays’ buffer must be contiguous.
Returns:
arraynumpy.ndarray
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, 5, 100]]) n_legs.to_numpy() array([ 2, 2, 4, 4, 5, 100])
to_pandas(self, memory_pool=None, categories=None, bool strings_to_categorical=False, bool zero_copy_only=False, bool integer_object_nulls=False, bool date_as_object=True, bool timestamp_as_object=False, bool use_threads=True, bool deduplicate_objects=True, bool ignore_metadata=False, bool safe=True, bool split_blocks=False, bool self_destruct=False, str maps_as_pydicts=None, types_mapper=None, bool coerce_temporal_nanoseconds=False)#
Convert to a pandas-compatible NumPy array or DataFrame, as appropriate
Parameters:
memory_poolMemoryPool, default None
Arrow MemoryPool to use for allocations. Uses the default memory pool if not passed.
categorieslist, default empty
List of fields that should be returned as pandas.Categorical. Only applies to table-like data structures.
strings_to_categoricalbool, default False
Encode string (UTF8) and binary types to pandas.Categorical.
zero_copy_onlybool, default False
Raise an ArrowException if this function call would require copying the underlying data.
integer_object_nullsbool, default False
Cast integers with nulls to objects
date_as_objectbool, default True
Cast dates to objects. If False, convert to datetime64 dtype with the equivalent time unit (if supported). Note: in pandas version < 2.0, only datetime64[ns] conversion is supported.
timestamp_as_objectbool, default False
Cast non-nanosecond timestamps (np.datetime64) to objects. This is useful in pandas version 1.x if you have timestamps that don’t fit in the normal date range of nanosecond timestamps (1678 CE-2262 CE). Non-nanosecond timestamps are supported in pandas version 2.0. If False, all timestamps are converted to datetime64 dtype.
Whether to parallelize the conversion using multiple threads.
deduplicate_objectsbool, default True
Do not create multiple copies Python objects when created, to save on memory use. Conversion will be slower.
ignore_metadatabool, default False
If True, do not use the ‘pandas’ metadata to reconstruct the DataFrame index, if present
For certain data types, a cast is needed in order to store the data in a pandas DataFrame or Series (e.g. timestamps are always stored as nanoseconds in pandas). This option controls whether it is a safe cast or not.
split_blocksbool, default False
If True, generate one internal “block” for each column when creating a pandas.DataFrame from a RecordBatch or Table. While this can temporarily reduce memory note that various pandas operations can trigger “consolidation” which may balloon memory use.
self_destructbool, default False
EXPERIMENTAL: If True, attempt to deallocate the originating Arrow memory while converting the Arrow object to pandas. If you use the object after calling to_pandas with this option it will crash your program.
Note that you may not see always memory usage improvements. For example, if multiple columns share an underlying allocation, memory can’t be freed until all columns are converted.
maps_as_pydictsstr, optional, default None
Valid values are None, ‘lossy’, or ‘strict’. The default behavior (None), is to convert Arrow Map arrays to Python association lists (list-of-tuples) in the same order as the Arrow Map, as in [(key1, value1), (key2, value2), …].
If ‘lossy’ or ‘strict’, convert Arrow Map arrays to native Python dicts. This can change the ordering of (key, value) pairs, and will deduplicate multiple keys, resulting in a possible loss of data.
If ‘lossy’, this key deduplication results in a warning printed when detected. If ‘strict’, this instead results in an exception being raised when detected.
types_mapperfunction, default None
A function mapping a pyarrow DataType to a pandas ExtensionDtype. This can be used to override the default pandas type for conversion of built-in pyarrow types or in absence of pandas_metadata in the Table schema. The function receives a pyarrow DataType and is expected to return a pandas ExtensionDtype or None if the default conversion should be used for that type. If you have a dictionary mapping, you can pass dict.get as function.
coerce_temporal_nanosecondsbool, default False
Only applicable to pandas version >= 2.0. A legacy option to coerce date32, date64, duration, and timestamp time units to nanoseconds when converting to pandas. This is the default behavior in pandas version 1.x. Set this option to True if you’d like to use this coercion when using pandas version >= 2.0 for backwards compatibility (not recommended otherwise).
Returns:
pandas.Series or pandas.DataFrame depending on type of object
Examples
import pyarrow as pa import pandas as pd
Convert a Table to pandas DataFrame:
table = pa.table([ ... pa.array([2, 4, 5, 100]), ... pa.array(["Flamingo", "Horse", "Brittle stars", "Centipede"]) ... ], names=['n_legs', 'animals']) table.to_pandas() n_legs animals 0 2 Flamingo 1 4 Horse 2 5 Brittle stars 3 100 Centipede isinstance(table.to_pandas(), pd.DataFrame) True
Convert a RecordBatch to pandas DataFrame:
import pyarrow as pa n_legs = pa.array([2, 4, 5, 100]) animals = pa.array(["Flamingo", "Horse", "Brittle stars", "Centipede"]) batch = pa.record_batch([n_legs, animals], ... names=["n_legs", "animals"]) batch pyarrow.RecordBatch n_legs: int64 animals: string
n_legs: [2,4,5,100] animals: ["Flamingo","Horse","Brittle stars","Centipede"]
batch.to_pandas() n_legs animals 0 2 Flamingo 1 4 Horse 2 5 Brittle stars 3 100 Centipede isinstance(batch.to_pandas(), pd.DataFrame) True
Convert a Chunked Array to pandas Series:
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, 5, 100]]) n_legs.to_pandas() 0 2 1 2 2 4 3 4 4 5 5 100 dtype: int64 isinstance(n_legs.to_pandas(), pd.Series) True
to_pylist(self, *, maps_as_pydicts=None)#
Convert to a list of native Python objects.
Parameters:
maps_as_pydictsstr, optional, default None
Valid values are None, ‘lossy’, or ‘strict’. The default behavior (None), is to convert Arrow Map arrays to Python association lists (list-of-tuples) in the same order as the Arrow Map, as in [(key1, value1), (key2, value2), …].
If ‘lossy’ or ‘strict’, convert Arrow Map arrays to native Python dicts.
If ‘lossy’, whenever duplicate keys are detected, a warning will be printed. The last seen value of a duplicate key will be in the Python dictionary. If ‘strict’, this instead results in an exception being raised when detected.
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, None, 100]]) n_legs.to_pylist() [2, 2, 4, 4, None, 100]
to_string(self, *, int indent=0, int window=5, int container_window=2, bool skip_new_lines=False, int element_size_limit=100)#
Render a “pretty-printed” string representation of the ChunkedArray
Parameters:
indentint
How much to indent right the content of the array, by default 0.
windowint
How many items to preview within each chunk at the begin and end of the chunk when the chunk is bigger than the window. The other elements will be ellipsed.
container_windowint
How many chunks to preview at the begin and end of the array when the array is bigger than the window. The other elements will be ellipsed. This setting also applies to list columns.
skip_new_linesbool
If the array should be rendered as a single line of text or if each element should be on its own line.
element_size_limitint, default 100
Maximum number of characters of a single element before it is truncated.
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, 5, 100]]) n_legs.to_string(skip_new_lines=True) '[[2,2,4],[4,5,100]]'
type#
Return data type of a ChunkedArray.
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, 5, 100]]) n_legs.type DataType(int64)
unify_dictionaries(self, MemoryPool memory_pool=None)#
Unify dictionaries across all chunks.
This method returns an equivalent chunked array, but where all chunks share the same dictionary values. Dictionary indices are transposed accordingly.
If there are no dictionaries in the chunked array, it is returned unchanged.
Parameters:
memory_poolMemoryPool, default None
For memory allocations, if required, otherwise use default pool
Returns:
resultChunkedArray
Examples
import pyarrow as pa arr_1 = pa.array(["Flamingo", "Parrot", "Dog"]).dictionary_encode() arr_2 = pa.array(["Horse", "Brittle stars", "Centipede"]).dictionary_encode() c_arr = pa.chunked_array([arr_1, arr_2]) c_arr <pyarrow.lib.ChunkedArray object at ...> [ ... -- dictionary: [ "Flamingo", "Parrot", "Dog" ] -- indices: [ 0, 1, 2 ], ... -- dictionary: [ "Horse", "Brittle stars", "Centipede" ] -- indices: [ 0, 1, 2 ] ] c_arr.unify_dictionaries() <pyarrow.lib.ChunkedArray object at ...> [ ... -- dictionary: [ "Flamingo", "Parrot", "Dog", "Horse", "Brittle stars", "Centipede" ] -- indices: [ 0, 1, 2 ], ... -- dictionary: [ "Flamingo", "Parrot", "Dog", "Horse", "Brittle stars", "Centipede" ] -- indices: [ 3, 4, 5 ] ]
unique(self)#
Compute distinct elements in array
Returns:
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, 5, 100]]) n_legs <pyarrow.lib.ChunkedArray object at ...> [ [ 2, 2, 4 ], [ 4, 5, 100 ] ] n_legs.unique() <pyarrow.lib.Int64Array object at ...> [ 2, 4, 5, 100 ]
validate(self, *, full=False)#
Perform validation checks. An exception is raised if validation fails.
By default only cheap validation checks are run. Pass full=Truefor thorough validation checks (potentially O(n)).
Parameters:
If True, run expensive checks, otherwise cheap checks only.
Raises:
ArrowInvalid
value_counts(self)#
Compute counts of unique elements in array.
Returns:
An array of <input type “Values”, int64_t “Counts”> structs
Examples
import pyarrow as pa n_legs = pa.chunked_array([[2, 2, 4], [4, 5, 100]]) n_legs <pyarrow.lib.ChunkedArray object at ...> [ [ 2, 2, 4 ], [ 4, 5, 100 ] ] n_legs.value_counts() <pyarrow.lib.StructArray object at ...> -- is_valid: all not null -- child 0 type: int64 [ 2, 4, 5, 100 ] -- child 1 type: int64 [ 2, 2, 1, 1 ]