pyarrow.parquet.read_table — Apache Arrow v20.0.0 (original) (raw)
pyarrow.parquet.read_table(source, *, columns=None, use_threads=True, schema=None, use_pandas_metadata=False, read_dictionary=None, memory_map=False, buffer_size=0, partitioning='hive', filesystem=None, filters=None, ignore_prefixes=None, pre_buffer=True, coerce_int96_timestamp_unit=None, decryption_properties=None, thrift_string_size_limit=None, thrift_container_size_limit=None, page_checksum_verification=False)[source]#
Read a Table from Parquet format
Parameters:
sourcestr, pyarrow.NativeFile, or file-like object
If a string passed, can be a single file name or directory name. For file-like objects, only read a single file. Use pyarrow.BufferReader to read a file contained in a bytes or buffer-like object.
columnslist
If not None, only these columns will be read from the file. A column name may be a prefix of a nested field, e.g. ‘a’ will select ‘a.b’, ‘a.c’, and ‘a.d.e’. If empty, no columns will be read. Note that the table will still have the correct num_rows set despite having no columns.
Perform multi-threaded column reads.
schemaSchema, optional
Optionally provide the Schema for the parquet dataset, in which case it will not be inferred from the source.
use_pandas_metadatabool, default False
If True and file has custom pandas schema metadata, ensure that index columns are also loaded.
read_dictionarylist, default None
List of names or column paths (for nested types) to read directly as DictionaryArray. Only supported for BYTE_ARRAY storage. To read a flat column as dictionary-encoded pass the column name. For nested types, you must pass the full column “path”, which could be something like level1.level2.list.item. Refer to the Parquet file’s schema to obtain the paths.
If the source is a file path, use a memory map to read file, which can improve performance in some environments.
buffer_sizeint, default 0
If positive, perform read buffering when deserializing individual column chunks. Otherwise IO calls are unbuffered.
partitioningpyarrow.dataset.Partitioning or str or list of str, default “hive”
The partitioning scheme for a partitioned dataset. The default of “hive” assumes directory names with key=value pairs like “/year=2009/month=11”. In addition, a scheme like “/2009/11” is also supported, in which case you need to specify the field names or a full schema. See thepyarrow.dataset.partitioning()
function for more details.
filesystemFileSystem, default None
If nothing passed, will be inferred based on path. Path will try to be found in the local on-disk filesystem otherwise it will be parsed as an URI to determine the filesystem.
filterspyarrow.compute.Expression
or List
[Tuple
] or List
[List
[Tuple
]], default None
Rows which do not match the filter predicate will be removed from scanned data. Partition keys embedded in a nested directory structure will be exploited to avoid loading files at all if they contain no matching rows. Within-file level filtering and different partitioning schemes are supported.
Predicates are expressed using an Expression
or using the disjunctive normal form (DNF), like [[('x', '=', 0), ...], ...]
. DNF allows arbitrary boolean logical combinations of single column predicates. The innermost tuples each describe a single column predicate. The list of inner predicates is interpreted as a conjunction (AND), forming a more selective and multiple column predicate. Finally, the most outer list combines these filters as a disjunction (OR).
Predicates may also be passed as List[Tuple]. This form is interpreted as a single conjunction. To express OR in predicates, one must use the (preferred) List[List[Tuple]] notation.
Each tuple has format: (key
, op
, value
) and compares thekey
with the value
. The supported op
are: =
or ==
, !=
, <
, >
, <=
,>=
, in
and not in
. If the op
is in
or not in
, thevalue
must be a collection such as a list
, a set
or atuple
.
Examples:
Using the Expression
API:
import pyarrow.compute as pc pc.field('x') = 0 pc.field('y').isin(['a', 'b', 'c']) ~pc.field('y').isin({'a', 'b'})
Using the DNF format:
('x', '=', 0) ('y', 'in', ['a', 'b', 'c']) ('z', 'not in', {'a','b'})
ignore_prefixeslist, optional
Files matching any of these prefixes will be ignored by the discovery process. This is matched to the basename of a path. By default this is [‘.’, ‘_’]. Note that discovery happens only if a directory is passed as source.
Coalesce and issue file reads in parallel to improve performance on high-latency filesystems (e.g. S3). If True, Arrow will use a background I/O thread pool. If using a filesystem layer that itself performs readahead (e.g. fsspec’s S3FS), disable readahead for best results.
coerce_int96_timestamp_unitstr, default None
Cast timestamps that are stored in INT96 format to a particular resolution (e.g. ‘ms’). Setting to None is equivalent to ‘ns’ and therefore INT96 timestamps will be inferred as timestamps in nanoseconds.
decryption_propertiesFileDecryptionProperties
or None
File-level decryption properties. The decryption properties can be created usingCryptoFactory.file_decryption_properties()
.
thrift_string_size_limitint, default None
If not None, override the maximum total string size allocated when decoding Thrift structures. The default limit should be sufficient for most Parquet files.
thrift_container_size_limitint, default None
If not None, override the maximum total size of containers allocated when decoding Thrift structures. The default limit should be sufficient for most Parquet files.
page_checksum_verificationbool, default False
If True, verify the checksum for each page read from the file.
Returns:
Content of the file as a table (of columns)
Examples
Generate an example PyArrow Table and write it to a partitioned dataset:
import pyarrow as pa table = pa.table({'year': [2020, 2022, 2021, 2022, 2019, 2021], ... 'n_legs': [2, 2, 4, 4, 5, 100], ... 'animal': ["Flamingo", "Parrot", "Dog", "Horse", ... "Brittle stars", "Centipede"]}) import pyarrow.parquet as pq pq.write_to_dataset(table, root_path='dataset_name_2', ... partition_cols=['year'])
Read the data:
pq.read_table('dataset_name_2').to_pandas() n_legs animal year 0 5 Brittle stars 2019 1 2 Flamingo 2020 2 4 Dog 2021 3 100 Centipede 2021 4 2 Parrot 2022 5 4 Horse 2022
Read only a subset of columns:
pq.read_table('dataset_name_2', columns=["n_legs", "animal"]) pyarrow.Table n_legs: int64 animal: string
n_legs: [[5],[2],[4,100],[2,4]] animal: [["Brittle stars"],["Flamingo"],["Dog","Centipede"],["Parrot","Horse"]]
Read a subset of columns and read one column as DictionaryArray:
pq.read_table('dataset_name_2', columns=["n_legs", "animal"], ... read_dictionary=["animal"]) pyarrow.Table n_legs: int64 animal: dictionary<values=string, indices=int32, ordered=0>
n_legs: [[5],[2],[4,100],[2,4]] animal: [ -- dictionary: ["Brittle stars"] -- indices: [0], -- dictionary: ["Flamingo"] -- indices: [0], -- dictionary: ["Dog","Centipede"] -- indices: [0,1], -- dictionary: ["Parrot","Horse"] -- indices: [0,1]]
Read the table with filter:
pq.read_table('dataset_name_2', columns=["n_legs", "animal"], ... filters=[('n_legs','<',4)]).to_pandas() n_legs animal 0 2 Flamingo 1 2 Parrot
Read data from a single Parquet file:
pq.write_table(table, 'example.parquet') pq.read_table('dataset_name_2').to_pandas() n_legs animal year 0 5 Brittle stars 2019 1 2 Flamingo 2020 2 4 Dog 2021 3 100 Centipede 2021 4 2 Parrot 2022 5 4 Horse 2022