Reading and Writing the Apache Parquet Format¶
The Apache Parquet project provides a standardized open-source columnar storage format for use in data analysis systems. It was created originally for use in Apache Hadoop with systems like Apache Drill, Apache Hive, Apache Impala (incubating), and Apache Spark adopting it as a shared standard for high performance data IO.
Apache Arrow is an ideal in-memory transport layer for data that is being read or written with Parquet files. We have been concurrently developing the C++ implementation of Apache Parquet, which includes a native, multithreaded C++ adapter to and from in-memory Arrow data. PyArrow includes Python bindings to this code, which thus enables reading and writing Parquet files with pandas as well.
Obtaining pyarrow with Parquet Support¶
If you installed pyarrow
with pip or conda, it should be built with Parquet
support bundled:
In [1]: import pyarrow.parquet as pq
If you are building pyarrow
from source, you must use
-DARROW_PARQUET=ON
when compiling the C++ libraries and enable the Parquet
extensions when building pyarrow
. See the Python Development page for more details.
Reading and Writing Single Files¶
The functions read_table()
and write_table()
read and write the pyarrow.Table object, respectively.
Let’s look at a simple table:
In [2]: import numpy as np
In [3]: import pandas as pd
In [4]: import pyarrow as pa
In [5]: df = pd.DataFrame({'one': [-1, np.nan, 2.5],
...: 'two': ['foo', 'bar', 'baz'],
...: 'three': [True, False, True]},
...: index=list('abc'))
...:
In [6]: table = pa.Table.from_pandas(df)
We write this to Parquet format with write_table
:
In [7]: import pyarrow.parquet as pq
In [8]: pq.write_table(table, 'example.parquet')
This creates a single Parquet file. In practice, a Parquet dataset may consist
of many files in many directories. We can read a single file back with
read_table
:
In [9]: table2 = pq.read_table('example.parquet')
In [10]: table2.to_pandas()
Out[10]:
one two three
a -1.0 foo True
b NaN bar False
c 2.5 baz True
You can pass a subset of columns to read, which can be much faster than reading the whole file (due to the columnar layout):
In [11]: pq.read_table('example.parquet', columns=['one', 'three'])
Out[11]:
pyarrow.Table
one: double
three: bool
When reading a subset of columns from a file that used a Pandas dataframe as the
source, we use read_pandas
to maintain any additional index column data:
In [12]: pq.read_pandas('example.parquet', columns=['two']).to_pandas()
Out[12]:
two
a foo
b bar
c baz
We need not use a string to specify the origin of the file. It can be any of:
A file path as a string
A NativeFile from PyArrow
A Python file object
In general, a Python file object will have the worst read performance, while a
string file path or an instance of NativeFile
(especially memory
maps) will perform the best.
Parquet file writing options¶
write_table()
has a number of options to
control various settings when writing a Parquet file.
version
, the Parquet format version to use, whether'1.0'
for compatibility with older readers, or'2.0'
to unlock more recent features.data_page_size
, to control the approximate size of encoded data pages within a column chunk. This currently defaults to 1MBflavor
, to set compatibility options particular to a Parquet consumer like'spark'
for Apache Spark.
See the write_table()
docstring for more details.
There are some additional data type handling-specific options described below.
Omitting the DataFrame index¶
When using pa.Table.from_pandas
to convert to an Arrow table, by default
one or more special columns are added to keep track of the index (row
labels). Storing the index takes extra space, so if your index is not valuable,
you may choose to omit it by passing preserve_index=False
In [13]: df = pd.DataFrame({'one': [-1, np.nan, 2.5],
....: 'two': ['foo', 'bar', 'baz'],
....: 'three': [True, False, True]},
....: index=list('abc'))
....:
In [14]: df
Out[14]:
one two three
a -1.0 foo True
b NaN bar False
c 2.5 baz True
In [15]: table = pa.Table.from_pandas(df, preserve_index=False)
Then we have:
In [16]: pq.write_table(table, 'example_noindex.parquet')
In [17]: t = pq.read_table('example_noindex.parquet')
In [18]: t.to_pandas()
Out[18]:
one two three
0 -1.0 foo True
1 NaN bar False
2 2.5 baz True
Here you see the index did not survive the round trip.
Finer-grained Reading and Writing¶
read_table
uses the ParquetFile
class, which has other features:
In [19]: parquet_file = pq.ParquetFile('example.parquet')
In [20]: parquet_file.metadata
Out[20]:
<pyarrow._parquet.FileMetaData object at 0x7efd79877360>
created_by: parquet-cpp version 1.5.1-SNAPSHOT
num_columns: 4
num_rows: 3
num_row_groups: 1
format_version: 1.0
serialized_size: 2636
In [21]: parquet_file.schema
Out[21]:
<pyarrow._parquet.ParquetSchema object at 0x7efd79a2ce08>
required group field_id=0 schema {
optional double field_id=1 one;
optional binary field_id=2 two (String);
optional boolean field_id=3 three;
optional binary field_id=4 __index_level_0__ (String);
}
As you can learn more in the Apache Parquet format, a Parquet file consists of
multiple row groups. read_table
will read all of the row groups and
concatenate them into a single table. You can read individual row groups with
read_row_group
:
In [22]: parquet_file.num_row_groups
Out[22]: 1
In [23]: parquet_file.read_row_group(0)
Out[23]:
pyarrow.Table
one: double
two: string
three: bool
__index_level_0__: string
We can similarly write a Parquet file with multiple row groups by using
ParquetWriter
:
In [24]: writer = pq.ParquetWriter('example2.parquet', table.schema)
In [25]: for i in range(3):
....: writer.write_table(table)
....:
In [26]: writer.close()
In [27]: pf2 = pq.ParquetFile('example2.parquet')
In [28]: pf2.num_row_groups
Out[28]: 3
Alternatively python with
syntax can also be use:
In [29]: with pq.ParquetWriter('example3.parquet', table.schema) as writer:
....: for i in range(3):
....: writer.write_table(table)
....:
Inspecting the Parquet File Metadata¶
The FileMetaData
of a Parquet file can be accessed through
ParquetFile
as shown above:
In [30]: parquet_file = pq.ParquetFile('example.parquet')
In [31]: metadata = parquet_file.metadata
or can also be read directly using read_metadata()
:
In [32]: metadata = pq.read_metadata('example.parquet')
In [33]: metadata
Out[33]:
<pyarrow._parquet.FileMetaData object at 0x7efd79877728>
created_by: parquet-cpp version 1.5.1-SNAPSHOT
num_columns: 4
num_rows: 3
num_row_groups: 1
format_version: 1.0
serialized_size: 2636
The returned FileMetaData
object allows to inspect the
Parquet file metadata,
such as the row groups and column chunk metadata and statistics:
In [34]: metadata.row_group(0)
Out[34]:
<pyarrow._parquet.RowGroupMetaData object at 0x7efd7982ee08>
num_columns: 4
num_rows: 3
total_byte_size: 296
In [35]: metadata.row_group(0).column(0)
Out[35]:
<pyarrow._parquet.ColumnChunkMetaData object at 0x7efd7878d188>
file_offset: 108
file_path:
physical_type: DOUBLE
num_values: 3
path_in_schema: one
is_stats_set: True
statistics:
<pyarrow._parquet.Statistics object at 0x7efd7878d228>
has_min_max: True
min: -1.0
max: 2.5
null_count: 1
distinct_count: 0
num_values: 2
physical_type: DOUBLE
logical_type: None
converted_type (legacy): NONE
compression: SNAPPY
encodings: ('PLAIN_DICTIONARY', 'PLAIN', 'RLE')
has_dictionary_page: True
dictionary_page_offset: 4
data_page_offset: 36
total_compressed_size: 104
total_uncompressed_size: 100
Data Type Handling¶
Reading types as DictionaryArray¶
The read_dictionary
option in read_table
and ParquetDataset
will
cause columns to be read as DictionaryArray
, which will become
pandas.Categorical
when converted to pandas. This option is only valid for
string and binary column types, and it can yield significantly lower memory use
and improved performance for columns with many repeated string values.
pq.read_table(table, where, read_dictionary=['binary_c0', 'stringb_c2'])
Storing timestamps¶
Some Parquet readers may only support timestamps stored in millisecond
('ms'
) or microsecond ('us'
) resolution. Since pandas uses nanoseconds
to represent timestamps, this can occasionally be a nuisance. By default
(when writing version 1.0 Parquet files), the nanoseconds will be cast to
microseconds (‘us’).
In addition, We provide the coerce_timestamps
option to allow you to select
the desired resolution:
pq.write_table(table, where, coerce_timestamps='ms')
If a cast to a lower resolution value may result in a loss of data, by default
an exception will be raised. This can be suppressed by passing
allow_truncated_timestamps=True
:
pq.write_table(table, where, coerce_timestamps='ms',
allow_truncated_timestamps=True)
Timestamps with nanoseconds can be stored without casting when using the more recent Parquet format version 2.0:
pq.write_table(table, where, version='2.0')
However, many Parquet readers do not yet support this newer format version, and therefore the default is to write version 1.0 files. When compatibility across different processing frameworks is required, it is recommended to use the default version 1.0.
Older Parquet implementations use INT96
based storage of
timestamps, but this is now deprecated. This includes some older
versions of Apache Impala and Apache Spark. To write timestamps in
this format, set the use_deprecated_int96_timestamps
option to
True
in write_table
.
pq.write_table(table, where, use_deprecated_int96_timestamps=True)
Compression, Encoding, and File Compatibility¶
The most commonly used Parquet implementations use dictionary encoding when
writing files; if the dictionaries grow too large, then they “fall back” to
plain encoding. Whether dictionary encoding is used can be toggled using the
use_dictionary
option:
pq.write_table(table, where, use_dictionary=False)
The data pages within a column in a row group can be compressed after the encoding passes (dictionary, RLE encoding). In PyArrow we use Snappy compression by default, but Brotli, Gzip, and uncompressed are also supported:
pq.write_table(table, where, compression='snappy')
pq.write_table(table, where, compression='gzip')
pq.write_table(table, where, compression='brotli')
pq.write_table(table, where, compression='none')
Snappy generally results in better performance, while Gzip may yield smaller files.
These settings can also be set on a per-column basis:
pq.write_table(table, where, compression={'foo': 'snappy', 'bar': 'gzip'},
use_dictionary=['foo', 'bar'])
Partitioned Datasets (Multiple Files)¶
Multiple Parquet files constitute a Parquet dataset. These may present in a number of ways:
A list of Parquet absolute file paths
A directory name containing nested directories defining a partitioned dataset
A dataset partitioned by year and month may look like on disk:
dataset_name/
year=2007/
month=01/
0.parq
1.parq
...
month=02/
0.parq
1.parq
...
month=03/
...
year=2008/
month=01/
...
...
Writing to Partitioned Datasets¶
You can write a partitioned dataset for any pyarrow
file system that is a
file-store (e.g. local, HDFS, S3). The default behaviour when no filesystem is
added is to use the local filesystem.
# Local dataset write
pq.write_to_dataset(table, root_path='dataset_name',
partition_cols=['one', 'two'])
The root path in this case specifies the parent directory to which data will be saved. The partition columns are the column names by which to partition the dataset. Columns are partitioned in the order they are given. The partition splits are determined by the unique values in the partition columns.
To use another filesystem you only need to add the filesystem parameter, the
individual table writes are wrapped using with
statements so the
pq.write_to_dataset
function does not need to be.
# Remote file-system example
fs = pa.hdfs.connect(host, port, user=user, kerb_ticket=ticket_cache_path)
pq.write_to_dataset(table, root_path='dataset_name',
partition_cols=['one', 'two'], filesystem=fs)
Compatibility Note: if using pq.write_to_dataset
to create a table that
will then be used by HIVE then partition column values must be compatible with
the allowed character set of the HIVE version you are running.
Writing _metadata
and _common_medata
files¶
Some processing frameworks such as Spark or Dask (optionally) use _metadata
and _common_metadata
files with partitioned datasets.
Those files include information about the schema of the full dataset (for
_common_metadata
) and potentially all row group metadata of all files in the
partitioned dataset as well (for _metadata
). The actual files are
metadata-only Parquet files. Note this is not a Parquet standard, but a
convention set in practice by those frameworks.
Using those files can give a more efficient creation of a parquet Dataset, since it can use the stored schema and and file paths of all row groups, instead of inferring the schema and crawling the directories for all Parquet files (this is especially the case for filesystems where accessing files is expensive).
The write_to_dataset()
function does not automatically
write such metadata files, but you can use it to gather the metadata and
combine and write them manually:
# Write a dataset and collect metadata information of all written files
metadata_collector = []
pq.write_to_dataset(table, root_path, metadata_collector=metadata_collector)
# Write the ``_common_metadata`` parquet file without row groups statistics
pq.write_metadata(table.schema, root_path / '_common_metadata')
# Write the ``_metadata`` parquet file with row groups statistics of all files
pq.write_metadata(
table.schema, root_path / '_metadata',
metadata_collector=metadata_collector
)
When not using the write_to_dataset()
function, but
writing the individual files of the partitioned dataset using
write_table()
or ParquetWriter
,
the metadata_collector
keyword can also be used to collect the FileMetaData
of the written files. In this case, you need to ensure to set the file path
contained in the row group metadata yourself before combining the metadata, and
the schemas of all different files and collected FileMetaData objects should be
the same:
metadata_collector = []
pq.write_table(
table1, root_path / "year=2017/data1.parquet",
metadata_collector=metadata_collector
)
# set the file path relative to the root of the partitioned dataset
metadata_collector[-1].set_file_path("year=2017/data1.parquet")
# combine and write the metadata
metadata = metadata_collector[0]
for _meta in metadata_collector[1:]:
metadata.append_row_groups(_meta)
metadata.write_metadata_file(root_path / "_metadata")
# or use pq.write_metadata to combine and write in a single step
pq.write_metadata(
table1.schema, root_path / "_metadata",
metadata_collector=metadata_collector
)
Reading from Partitioned Datasets¶
The ParquetDataset
class accepts either a directory name or a list
or file paths, and can discover and infer some common partition structures,
such as those produced by Hive:
dataset = pq.ParquetDataset('dataset_name/')
table = dataset.read()
You can also use the convenience function read_table
exposed by
pyarrow.parquet
that avoids the need for an additional Dataset object
creation step.
table = pq.read_table('dataset_name')
Note: the partition columns in the original table will have their types
converted to Arrow dictionary types (pandas categorical) on load. Ordering of
partition columns is not preserved through the save/load process. If reading
from a remote filesystem into a pandas dataframe you may need to run
sort_index
to maintain row ordering (as long as the preserve_index
option was enabled on write).
Note
The ParquetDataset is being reimplemented based on the new generic Dataset
API (see the Tabular Datasets docs for an overview). This is not yet the
default, but can already be enabled by passing the use_legacy_dataset=False
keyword to ParquetDataset
or read_table()
:
pq.ParquetDataset('dataset_name/', use_legacy_dataset=False)
Enabling this gives the following new features:
Filtering on all columns (using row group statistics) instead of only on the partition keys.
More fine-grained partitioning: support for a directory partitioning scheme in addition to the Hive-like partitioning (e.g. “/2019/11/15/” instead of “/year=2019/month=11/day=15/”), and the ability to specify a schema for the partition keys.
General performance improvement and bug fixes.
It also has the following changes in behaviour:
The partition keys need to be explicitly included in the
columns
keyword when you want to include them in the result while reading a subset of the columns
This new implementation is already enabled in read_table
, and in the
future, this will be turned on by default for ParquetDataset
. The new
implementation does not yet cover all existing ParquetDataset features (e.g.
specifying the metadata
, or the pieces
property API). Feedback is
very welcome.
Using with Spark¶
Spark places some constraints on the types of Parquet files it will read. The
option flavor='spark'
will set these options automatically and also
sanitize field characters unsupported by Spark SQL.
Multithreaded Reads¶
Each of the reading functions by default use multi-threading for reading columns in parallel. Depending on the speed of IO and how expensive it is to decode the columns in a particular file (particularly with GZIP compression), this can yield significantly higher data throughput.
This can be disabled by specifying use_threads=False
.
Note
The number of threads to use concurrently is automatically inferred by Arrow
and can be inspected using the cpu_count()
function.
Reading a Parquet File from Azure Blob storage¶
The code below shows how to use Azure’s storage sdk along with pyarrow to read a parquet file into a Pandas dataframe. This is suitable for executing inside a Jupyter notebook running on a Python 3 kernel.
Dependencies:
python 3.6.2
azure-storage 0.36.0
pyarrow 0.8.0
import pyarrow.parquet as pq
from io import BytesIO
from azure.storage.blob import BlockBlobService
account_name = '...'
account_key = '...'
container_name = '...'
parquet_file = 'mysample.parquet'
byte_stream = io.BytesIO()
block_blob_service = BlockBlobService(account_name=account_name, account_key=account_key)
try:
block_blob_service.get_blob_to_stream(container_name=container_name, blob_name=parquet_file, stream=byte_stream)
df = pq.read_table(source=byte_stream).to_pandas()
# Do work on df ...
finally:
# Add finally block to ensure closure of the stream
byte_stream.close()
Notes:
The
account_key
can be found underSettings -> Access keys
in the Microsoft Azure portal for a given containerThe code above works for a container with private access, Lease State = Available, Lease Status = Unlocked
The parquet file was Blob Type = Block blob