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 objects, 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
metadata
--------
OrderedDict([(b'pandas',
              b'{"index_columns": [{"kind": "serialized", "field_name": "__i'
              b'ndex_level_0__"}], "column_indexes": [{"name": null, "field_'
              b'name": null, "pandas_type": "unicode", "numpy_type": "object'
              b'", "metadata": {"encoding": "UTF-8"}}], "columns": [{"name":'
              b' "one", "field_name": "one", "pandas_type": "float64", "nump'
              b'y_type": "float64", "metadata": null}, {"name": "two", "fiel'
              b'd_name": "two", "pandas_type": "unicode", "numpy_type": "obj'
              b'ect", "metadata": null}, {"name": "three", "field_name": "th'
              b'ree", "pandas_type": "bool", "numpy_type": "bool", "metadata'
              b'": null}, {"name": null, "field_name": "__index_level_0__", '
              b'"pandas_type": "unicode", "numpy_type": "object", "metadata"'
              b': null}], "creator": {"library": "pyarrow", "version": "0.12'
              b'.1.dev425+g828b4377f.d20190316"}, "pandas_version": "0.23.4"'
              b'}')])

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.

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 0x7fde3fc60908>
  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: 1167

In [21]: parquet_file.schema
Out[21]: 
<pyarrow._parquet.ParquetSchema object at 0x7fde3fc67390>
one: DOUBLE
two: BYTE_ARRAY UTF8
three: BOOLEAN
__index_level_0__: BYTE_ARRAY UTF8

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
metadata
--------
OrderedDict([(b'pandas',
              b'{"index_columns": [{"kind": "serialized", "field_name": "__i'
              b'ndex_level_0__"}], "column_indexes": [{"name": null, "field_'
              b'name": null, "pandas_type": "unicode", "numpy_type": "object'
              b'", "metadata": {"encoding": "UTF-8"}}], "columns": [{"name":'
              b' "one", "field_name": "one", "pandas_type": "float64", "nump'
              b'y_type": "float64", "metadata": null}, {"name": "two", "fiel'
              b'd_name": "two", "pandas_type": "unicode", "numpy_type": "obj'
              b'ect", "metadata": null}, {"name": "three", "field_name": "th'
              b'ree", "pandas_type": "bool", "numpy_type": "bool", "metadata'
              b'": null}, {"name": null, "field_name": "__index_level_0__", '
              b'"pandas_type": "unicode", "numpy_type": "object", "metadata"'
              b': null}], "creator": {"library": "pyarrow", "version": "0.12'
              b'.1.dev425+g828b4377f.d20190316"}, "pandas_version": "0.23.4"'
              b'}')])

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)
   ....: 

Data Type Handling

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. 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)

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.

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).

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 have an nthreads argument which will read columns with the indicated level of parallelism. 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:

pq.read_table(where, nthreads=4)
pq.ParquetDataset(where).read(nthreads=4)

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 under Settings -> Access keys in the Microsoft Azure portal for a given container
  • The code above works for a container with private access, Lease State = Available, Lease Status = Unlocked
  • The parquet file was Blob Type = Block blob