This is the multi-page printable view of this section. Click here to print.

Return to the regular view of this page.

Documentation

Welcome to the documentation for Apache Parquet. Here, you can find information about the Parquet File Format, including specifications and developer resources.

1 - Overview

All about Parquet.

Apache Parquet is a columnar storage format available to any project in the Hadoop ecosystem, regardless of the choice of data processing framework, data model or programming language.

1.1 - Motivation

We created Parquet to make the advantages of compressed, efficient columnar data representation available to any project in the Hadoop ecosystem.

Parquet is built from the ground up with complex nested data structures in mind, and uses the record shredding and assembly algorithm described in the Dremel paper. We believe this approach is superior to simple flattening of nested name spaces.

Parquet is built to support very efficient compression and encoding schemes. Multiple projects have demonstrated the performance impact of applying the right compression and encoding scheme to the data. Parquet allows compression schemes to be specified on a per-column level, and is future-proofed to allow adding more encodings as they are invented and implemented.

Parquet is built to be used by anyone. The Hadoop ecosystem is rich with data processing frameworks, and we are not interested in playing favorites. We believe that an efficient, well-implemented columnar storage substrate should be useful to all frameworks without the cost of extensive and difficult to set up dependencies.

2 - Concepts

Glossary of relevant terminology.

Block (hdfs block): This means a block in hdfs and the meaning is unchanged for describing this file format. The file format is designed to work well on top of hdfs.

File: A hdfs file that must include the metadata for the file. It does not need to actually contain the data.

Row group: A logical horizontal partitioning of the data into rows. There is no physical structure that is guaranteed for a row group. A row group consists of a column chunk for each column in the dataset.

Column chunk: A chunk of the data for a particular column. These live in a particular row group and is guaranteed to be contiguous in the file.

Page: Column chunks are divided up into pages. A page is conceptually an indivisible unit (in terms of compression and encoding). There can be multiple page types which is interleaved in a column chunk.

Hierarchically, a file consists of one or more row groups. A row group contains exactly one column chunk per column. Column chunks contain one or more pages.

Unit of parallelization

  • MapReduce - File/Row Group
  • IO - Column chunk
  • Encoding/Compression - Page

3 - File Format

Documentation about the Parquet File Format.

This file and the thrift definition should be read together to understand the format.

4-byte magic number "PAR1"
<Column 1 Chunk 1 + Column Metadata>
<Column 2 Chunk 1 + Column Metadata>
...
<Column N Chunk 1 + Column Metadata>
<Column 1 Chunk 2 + Column Metadata>
<Column 2 Chunk 2 + Column Metadata>
...
<Column N Chunk 2 + Column Metadata>
...
<Column 1 Chunk M + Column Metadata>
<Column 2 Chunk M + Column Metadata>
...
<Column N Chunk M + Column Metadata>
File Metadata
4-byte length in bytes of file metadata
4-byte magic number "PAR1"

In the above example, there are N columns in this table, split into M row groups. The file metadata contains the locations of all the column metadata start locations. More details on what is contained in the metadata can be found in the thrift files.

Metadata is written after the data to allow for single pass writing.

Readers are expected to first read the file metadata to find all the column chunks they are interested in. The columns chunks should then be read sequentially.

The format is explicitly designed to separate the metadata from the data. This allows splitting columns into multiple files, as well as having a single metadata file reference multiple parquet files.

File Layout

3.1 - Configurations

Row Group Size

Larger row groups allow for larger column chunks which makes it possible to do larger sequential IO. Larger groups also require more buffering in the write path (or a two pass write). We recommend large row groups (512MB - 1GB). Since an entire row group might need to be read, we want it to completely fit on one HDFS block. Therefore, HDFS block sizes should also be set to be larger. An optimized read setup would be: 1GB row groups, 1GB HDFS block size, 1 HDFS block per HDFS file.

Data Page Size

Data pages should be considered indivisible so smaller data pages allow for more fine grained reading (e.g. single row lookup). Larger page sizes incur less space overhead (less page headers) and potentially less parsing overhead (processing headers). Note: for sequential scans, it is not expected to read a page at a time; this is not the IO chunk. We recommend 8KB for page sizes.

3.2 - Extensibility

There are many places in the format for compatible extensions:

File Version: The file metadata contains a version. Encodings: Encodings are specified by enum and more can be added in the future. Page types: Additional page types can be added and safely skipped.

3.3 - Metadata

There are three types of metadata: file metadata, column (chunk) metadata and page header metadata. All thrift structures are serialized using the TCompactProtocol.

File Layout

3.4 - Types

The types supported by the file format are intended to be as minimal as possible, with a focus on how the types effect on disk storage. For example, 16-bit ints are not explicitly supported in the storage format since they are covered by 32-bit ints with an efficient encoding. This reduces the complexity of implementing readers and writers for the format. The types are:

BOOLEAN: 1 bit boolean
INT32: 32 bit signed ints
INT64: 64 bit signed ints
INT96: 96 bit signed ints
FLOAT: IEEE 32-bit floating point values
DOUBLE: IEEE 64-bit floating point values
BYTE_ARRAY: arbitrarily long byte arrays.

3.4.1 - Logical Types

Logical types are used to extend the types that parquet can be used to store, by specifying how the primitive types should be interpreted. This keeps the set of primitive types to a minimum and reuses parquet’s efficient encodings. For example, strings are stored as byte arrays (binary) with a UTF8 annotation. These annotations define how to further decode and interpret the data. Annotations are stored as a ConvertedType in the file metadata and are documented in LogicalTypes.md.

3.5 - Nested Encoding

To encode nested columns, Parquet uses the Dremel encoding with definition and repetition levels. Definition levels specify how many optional fields in the path for the column are defined. Repetition levels specify at what repeated field in the path has the value repeated. The max definition and repetition levels can be computed from the schema (i.e. how much nesting there is). This defines the maximum number of bits required to store the levels (levels are defined for all values in the column).

Two encodings for the levels are supported BITPACKED and RLE. Only RLE is now used as it supersedes BITPACKED.

3.6 - Data Pages

For data pages, the 3 pieces of information are encoded back to back, after the page header. We have the

  • definition levels data,
  • repetition levels data,
  • encoded values. The size of specified in the header is for all 3 pieces combined.

The data for the data page is always required. The definition and repetition levels are optional, based on the schema definition. If the column is not nested (i.e. the path to the column has length 1), we do not encode the repetition levels (it would always have the value 1). For data that is required, the definition levels are skipped (if encoded, it will always have the value of the max definition level).

For example, in the case where the column is non-nested and required, the data in the page is only the encoded values.

The supported encodings are described in Encodings.md

3.6.1 - Encodings

Plain: (PLAIN = 0)

Supported Types: all

This is the plain encoding that must be supported for types. It is intended to be the simplest encoding. Values are encoded back to back.

The plain encoding is used whenever a more efficient encoding can not be used. It stores the data in the following format:

  • BOOLEAN: Bit Packed, LSB first
  • INT32: 4 bytes little endian
  • INT64: 8 bytes little endian
  • INT96: 12 bytes little endian (deprecated)
  • FLOAT: 4 bytes IEEE little endian
  • DOUBLE: 8 bytes IEEE little endian
  • BYTE_ARRAY: length in 4 bytes little endian followed by the bytes contained in the array
  • FIXED_LEN_BYTE_ARRAY: the bytes contained in the array

For native types, this outputs the data as little endian. Floating point types are encoded in IEEE.

For the byte array type, it encodes the length as a 4 byte little endian, followed by the bytes.

Dictionary Encoding (PLAIN_DICTIONARY = 2 and RLE_DICTIONARY = 8)

The dictionary encoding builds a dictionary of values encountered in a given column. The dictionary will be stored in a dictionary page per column chunk. The values are stored as integers using the RLE/Bit-Packing Hybrid encoding. If the dictionary grows too big, whether in size or number of distinct values, the encoding will fall back to the plain encoding. The dictionary page is written first, before the data pages of the column chunk.

Dictionary page format: the entries in the dictionary - in dictionary order - using the plain encoding.

Data page format: the bit width used to encode the entry ids stored as 1 byte (max bit width = 32), followed by the values encoded using RLE/Bit packed described above (with the given bit width).

Using the PLAIN_DICTIONARY enum value is deprecated in the Parquet 2.0 specification. Prefer using RLE_DICTIONARY in a data page and PLAIN in a dictionary page for Parquet 2.0+ files.

Run Length Encoding / Bit-Packing Hybrid (RLE = 3)

This encoding uses a combination of bit-packing and run length encoding to more efficiently store repeated values.

The grammar for this encoding looks like this, given a fixed bit-width known in advance:

rle-bit-packed-hybrid: <length> <encoded-data>
length := length of the <encoded-data> in bytes stored as 4 bytes little endian (unsigned int32)
encoded-data := <run>*
run := <bit-packed-run> | <rle-run>
bit-packed-run := <bit-packed-header> <bit-packed-values>
bit-packed-header := varint-encode(<bit-pack-scaled-run-len> << 1 | 1)
// we always bit-pack a multiple of 8 values at a time, so we only store the number of values / 8
bit-pack-scaled-run-len := (bit-packed-run-len) / 8
bit-packed-run-len := *see 3 below*
bit-packed-values := *see 1 below*
rle-run := <rle-header> <repeated-value>
rle-header := varint-encode( (rle-run-len) << 1)
rle-run-len := *see 3 below*
repeated-value := value that is repeated, using a fixed-width of round-up-to-next-byte(bit-width)
  1. The bit-packing here is done in a different order than the one in the deprecated bit-packing encoding. The values are packed from the least significant bit of each byte to the most significant bit, though the order of the bits in each value remains in the usual order of most significant to least significant. For example, to pack the same values as the example in the deprecated encoding above:

    The numbers 1 through 7 using bit width 3:

    dec value: 0   1   2   3   4   5   6   7
    bit value: 000 001 010 011 100 101 110 111
    bit label: ABC DEF GHI JKL MNO PQR STU VWX
    

    would be encoded like this where spaces mark byte boundaries (3 bytes):

    bit value: 10001000 11000110 11111010
    bit label: HIDEFABC RMNOJKLG VWXSTUPQ
    

    The reason for this packing order is to have fewer word-boundaries on little-endian hardware when deserializing more than one byte at at time. This is because 4 bytes can be read into a 32 bit register (or 8 bytes into a 64 bit register) and values can be unpacked just by shifting and ORing with a mask. (to make this optimization work on a big-endian machine, you would have to use the ordering used in the deprecated bit-packing encoding)

  2. varint-encode() is ULEB-128 encoding, see https://en.wikipedia.org/wiki/LEB128

  3. bit-packed-run-len and rle-run-len must be in the range [1, 231 - 1]. This means that a Parquet implementation can always store the run length in a signed 32-bit integer. This length restriction was not part of the Parquet 2.5.0 and earlier specifications, but longer runs were not readable by the most common Parquet implementations so, in practice, were not safe for Parquet writers to emit.

Note that the RLE encoding method is only supported for the following types of data:

  • Repetition and definition levels
  • Dictionary indices
  • Boolean values in data pages, as an alternative to PLAIN encoding

Bit-packed (Deprecated) (BIT_PACKED = 4)

This is a bit-packed only encoding, which is deprecated and will be replaced by the RLE/bit-packing hybrid encoding. Each value is encoded back to back using a fixed width. There is no padding between values (except for the last byte) which is padded with 0s. For example, if the max repetition level was 3 (2 bits) and the max definition level as 3 (2 bits), to encode 30 values, we would have 30 * 2 = 60 bits = 8 bytes.

This implementation is deprecated because the RLE/bit-packing hybrid is a superset of this implementation. For compatibility reasons, this implementation packs values from the most significant bit to the least significant bit, which is not the same as the RLE/bit-packing hybrid.

For example, the numbers 1 through 7 using bit width 3:

dec value: 0   1   2   3   4   5   6   7
bit value: 000 001 010 011 100 101 110 111
bit label: ABC DEF GHI JKL MNO PQR STU VWX

would be encoded like this where spaces mark byte boundaries (3 bytes):

bit value: 00000101 00111001 01110111
bit label: ABCDEFGH IJKLMNOP QRSTUVWX

Note that the BIT_PACKED encoding method is only supported for encoding repetition and definition levels.

Delta Encoding (DELTA_BINARY_PACKED = 5)

Supported Types: INT32, INT64

This encoding is adapted from the Binary packing described in “Decoding billions of integers per second through vectorization” by D. Lemire and L. Boytsov.

In delta encoding we make use of variable length integers for storing various numbers (not the deltas themselves). For unsigned values, we use ULEB128, which is the unsigned version of LEB128 (https://en.wikipedia.org/wiki/LEB128#Unsigned_LEB128). For signed values, we use zigzag encoding (https://developers.google.com/protocol-buffers/docs/encoding#signed-integers) to map negative values to positive ones and apply ULEB128 on the result.

Delta encoding consists of a header followed by blocks of delta encoded values binary packed. Each block is made of miniblocks, each of them binary packed with its own bit width.

The header is defined as follows:

<block size in values> <number of miniblocks in a block> <total value count> <first value>
  • the block size is a multiple of 128; it is stored as a ULEB128 int
  • the miniblock count per block is a divisor of the block size such that their quotient, the number of values in a miniblock, is a multiple of 32; it is stored as a ULEB128 int
  • the total value count is stored as a ULEB128 int
  • the first value is stored as a zigzag ULEB128 int

Each block contains

<min delta> <list of bitwidths of miniblocks> <miniblocks>
  • the min delta is a zigzag ULEB128 int (we compute a minimum as we need positive integers for bit packing)
  • the bitwidth of each block is stored as a byte
  • each miniblock is a list of bit packed ints according to the bit width stored at the begining of the block

To encode a block, we will:

  1. Compute the differences between consecutive elements. For the first element in the block, use the last element in the previous block or, in the case of the first block, use the first value of the whole sequence, stored in the header.

  2. Compute the frame of reference (the minimum of the deltas in the block). Subtract this min delta from all deltas in the block. This guarantees that all values are non-negative.

  3. Encode the frame of reference (min delta) as a zigzag ULEB128 int followed by the bit widths of the miniblocks and the delta values (minus the min delta) bit packed per miniblock.

Having multiple blocks allows us to adapt to changes in the data by changing the frame of reference (the min delta) which can result in smaller values after the subtraction which, again, means we can store them with a lower bit width.

If there are not enough values to fill the last miniblock, we pad the miniblock so that its length is always the number of values in a full miniblock multiplied by the bit width. The values of the padding bits should be zero, but readers must accept paddings consisting of arbitrary bits as well.

If, in the last block, less than <number of miniblocks in a block> miniblocks are needed to store the values, the bytes storing the bit widths of the unneeded miniblocks are still present, their value should be zero, but readers must accept arbitrary values as well. There are no additional padding bytes for the miniblock bodies though, as if their bit widths were 0 (regardless of the actual byte values). The reader knows when to stop reading by keeping track of the number of values read.

The following examples use 8 as the block size to keep the examples short, but in real cases it would be invalid.

Example 1

1, 2, 3, 4, 5

After step 1), we compute the deltas as:

1, 1, 1, 1

The minimum delta is 1 and after step 2, the deltas become

0, 0, 0, 0

The final encoded data is:

header: 8 (block size), 1 (miniblock count), 5 (value count), 1 (first value)

block 1 (minimum delta), 0 (bitwidth), (no data needed for bitwidth 0)

Example 2

7, 5, 3, 1, 2, 3, 4, 5, the deltas would be

-2, -2, -2, 1, 1, 1, 1

The minimum is -2, so the relative deltas are:

0, 0, 0, 3, 3, 3, 3

The encoded data is

header: 8 (block size), 1 (miniblock count), 8 (value count), 7 (first value)

block -2 (minimum delta), 2 (bitwidth), 00000011111111b (0,0,0,3,3,3,3 packed on 2 bits)

Characteristics

This encoding is similar to the RLE/bit-packing encoding. However the RLE/bit-packing encoding is specifically used when the range of ints is small over the entire page, as is true of repetition and definition levels. It uses a single bit width for the whole page. The delta encoding algorithm described above stores a bit width per miniblock and is less sensitive to variations in the size of encoded integers. It is also somewhat doing RLE encoding as a block containing all the same values will be bit packed to a zero bit width thus being only a header.

Delta-length byte array: (DELTA_LENGTH_BYTE_ARRAY = 6)

Supported Types: BYTE_ARRAY

This encoding is always preferred over PLAIN for byte array columns.

For this encoding, we will take all the byte array lengths and encode them using delta encoding (DELTA_BINARY_PACKED). The byte array data follows all of the length data just concatenated back to back. The expected savings is from the cost of encoding the lengths and possibly better compression in the data (it is no longer interleaved with the lengths).

The data stream looks like:

For example, if the data was “Hello”, “World”, “Foobar”, “ABCDEF”:

The encoded data would be DeltaEncoding(5, 5, 6, 6) “HelloWorldFoobarABCDEF”

Delta Strings: (DELTA_BYTE_ARRAY = 7)

Supported Types: BYTE_ARRAY

This is also known as incremental encoding or front compression: for each element in a sequence of strings, store the prefix length of the previous entry plus the suffix.

For a longer description, see https://en.wikipedia.org/wiki/Incremental_encoding.

This is stored as a sequence of delta-encoded prefix lengths (DELTA_BINARY_PACKED), followed by the suffixes encoded as delta length byte arrays (DELTA_LENGTH_BYTE_ARRAY).

Byte Stream Split: (BYTE_STREAM_SPLIT = 9)

Supported Types: FLOAT DOUBLE

This encoding does not reduce the size of the data but can lead to a significantly better compression ratio and speed when a compression algorithm is used afterwards.

This encoding creates K byte-streams of length N where K is the size in bytes of the data type and N is the number of elements in the data sequence. The bytes of each value are scattered to the corresponding streams. The 0-th byte goes to the 0-th stream, the 1-st byte goes to the 1-st stream and so on. The streams are concatenated in the following order: 0-th stream, 1-st stream, etc.

Example: Original data is three 32-bit floats and for simplicity we look at their raw representation.

       Element 0      Element 1      Element 2
Bytes  AA BB CC DD    00 11 22 33    A3 B4 C5 D6

After applying the transformation, the data has the following representation:

Bytes  AA 00 A3 BB 11 B4 CC 22 C5 DD 33 D6

3.6.2 - Checksumming

Column chunks are composed of pages written back to back. The pages share a common header and readers can skip over page they are not interested in. The data for the page follows the header and can be compressed and/or encoded. The compression and encoding is specified in the page metadata.

3.6.3 - Column Chunks

Column chunks are composed of pages written back to back. The pages share a common header and readers can skip over page they are not interested in. The data for the page follows the header and can be compressed and/or encoded. The compression and encoding is specified in the page metadata.

3.6.4 - Error Recovery

If the file metadata is corrupt, the file is lost. If the column metadata is corrupt, that column chunk is lost (but column chunks for this column in other row groups are okay). If a page header is corrupt, the remaining pages in that chunk are lost. If the data within a page is corrupt, that page is lost. The file will be more resilient to corruption with smaller row groups.

Potential extension: With smaller row groups, the biggest issue is placing the file metadata at the end. If an error happens while writing the file metadata, all the data written will be unreadable. This can be fixed by writing the file metadata every Nth row group. Each file metadata would be cumulative and include all the row groups written so far. Combining this with the strategy used for orc or avro files using sync markers, a reader could recover partially written files.

3.7 - Nulls

Nullity is encoded in the definition levels (which is run-length encoded). NULL values are not encoded in the data. For example, in a non-nested schema, a column with 1000 NULLs would be encoded with run-length encoding (0, 1000 times) for the definition levels and nothing else.

4 - Developer Guide

All developer resources related to Parquet.

This section contains the developer specific documentation related to Parquet.

4.1 - Modules

The parquet-format project contains format specifications and Thrift definitions of metadata required to properly read Parquet files.

The parquet-mr project contains multiple sub-modules, which implement the core components of reading and writing a nested, column-oriented data stream, map this core onto the parquet format, and provide Hadoop Input/Output Formats, Pig loaders, and other Java-based utilities for interacting with Parquet.

The parquet-cpp project is a C++ library to read-write Parquet files. It is part of the Apache Arrow C++ implementation, with bindings to Python, R, Ruby and C/GLib.

The parquet-rs project is a Rust library to read-write Parquet files.

The parquet-compatibility project (deprecated) contains compatibility tests that can be used to verify that implementations in different languages can read and write each other’s files. As of January 2022 compatibility tests only exist up to version 1.2.0.

4.2 - Building Parquet

How to build Parquet

Building Java resources can be build using mvn package. The current stable version should always be available from Maven Central.

C++ thrift resources can be generated via make.

Thrift can be also code-genned into any other thrift-supported language.

4.3 - Contributing to Parquet

How to contribute to Parquet

Pull Requests

We prefer to receive contributions in the form of GitHub pull requests. Please send pull requests against the github.com/apache/parquet-mr repository. If you’ve previously forked Parquet from its old location, you will need to add a remote or update your origin remote to https://github.com/apache/parquet-mr.git Here are a few tips to get your contribution in:

  1. Break your work into small, single-purpose patches if possible. It’s much harder to merge in a large change with a lot of disjoint features.
  2. Create a JIRA for your patch on the Parquet Project JIRA.
  3. Submit the patch as a GitHub pull request against the master branch. For a tutorial, see the GitHub guides on forking a repo and sending a pull request. Prefix your pull request name with the JIRA name (ex: https://github.com/apache/parquet-mr/pull/5).
  4. Make sure that your code passes the unit tests. You can run the tests with mvn test in the root directory.
  5. Add new unit tests for your code.
  6. All Pull Requests are tested automatically on GitHub Actions. TravisCI is also used to run the tests on ARM64 CPU architecture

If you’d like to report a bug but don’t have time to fix it, you can still post it to our issue tracker, or email the mailing list (dev@parquet.apache.org).

Committers

Merging a pull request requires being a comitter on the project.

How to merge a Pull request (have an apache and github-apache remote setup):

git remote add github-apache git@github.com:apache/parquet-mr.git
git remote add apache https://gitbox.apache.org/repos/asf?p=parquet-mr.git

run the following command

dev/merge_parquet_pr.py

example output:

Which pull request would you like to merge? (e.g. 34):

Type the pull request number (from https://github.com/apache/parquet-mr/pulls) and hit enter.

=== Pull Request #X ===
title   Blah Blah Blah
source  repo/branch
target  master
url https://api.github.com/repos/apache/parquet-mr/pulls/X

Proceed with merging pull request #3? (y/n):

If this looks good, type y and hit enter.

From gitbox.apache.org:/repos/asf/parquet-mr.git
* [new branch]      master     -> PR_TOOL_MERGE_PR_3_MASTER
Switched to branch 'PR_TOOL_MERGE_PR_3_MASTER'

Merge complete (local ref PR_TOOL_MERGE_PR_3_MASTER). Push to apache? (y/n):

A local branch with the merge has been created. Type y and hit enter to push it to apache master

Counting objects: 67, done.
Delta compression using up to 4 threads.
Compressing objects: 100% (26/26), done.
Writing objects: 100% (36/36), 5.32 KiB, done.
Total 36 (delta 17), reused 0 (delta 0)
To gitbox.apache.org:/repos/asf/parquet-mr.git
   b767ac4..485658a  PR_TOOL_MERGE_PR_X_MASTER -> master
Restoring head pointer to b767ac4e
Note: checking out 'b767ac4e'.

You are in 'detached HEAD' state. You can look around, make experimental
changes and commit them, and you can discard any commits you make in this
state without impacting any branches by performing another checkout.

If you want to create a new branch to retain commits you create, you may
do so (now or later) by using -b with the checkout command again. Example:

  git checkout -b new_branch_name

HEAD is now at b767ac4... Update README.md
Deleting local branch PR_TOOL_MERGE_PR_X
Deleting local branch PR_TOOL_MERGE_PR_X_MASTER
Pull request #X merged!
Merge hash: 485658a5

Would you like to pick 485658a5 into another branch? (y/n):

For now just say n as we have 1 branch

Website

Release Documentation

To create documentation for a new release of parquet-format create a new .md file under content/en/blog/parquet-format. Please see existing files in that directory as an example.

To create documentation for a new release of parquet-mr create a new .md file under content/en/blog/parquet-mr. Please see existing files in that directory as an example.

Website development and deployment

Staging

To make a change to the staging version of the website:

  1. Make a PR against the staging branch in the repository
  2. Once the PR is merged, the Build and Deploy Parquet Site job in the deployment workflow will be run, populating the asf-staging branch on this repo with the necessary files.

Do not directly edit the asf-staging branch of this repo

Production

To make a change to the production version of the website:

  1. Make a PR against the production branch in the repository
  2. Once the PR is merged, the Build and Deploy Parquet Site job in the deployment workflow will be run, populating the asf-site branch on this repo with the necessary files.

Do not directly edit the asf-site branch of this repo

4.4 - Releasing Parquet

How to release Parquet

Setup

You will need: * PGP code signing keys, published in KEYS * Permission to stage artifacts in Nexus

Make sure you have permission to deploy Parquet artifacts to Nexus by pushing a snapshot:

mvn deploy

If you have problems, read the publishing Maven artifacts documentation

Release process

Parquet uses the maven-release-plugin to tag a release and push binary artifacts to staging in Nexus. Once maven completes the release, the offical source tarball is built from the tag.

Before you start the release process:

  1. Verify that the release is finished (no planned JIRAs are pending and all patches are cherry-picked to the release branch)
  2. Resolve all associated JIRAs with correct target version and create the next unreleased version in the JIRA project
  3. Build and test the project
  4. Create a new branch for the release if this is a new minor version. For example, if the new minor version is 1.13.0, create a new branch parquet-1.13.x
  5. Update the change log
    • Go to the release notes for the release in JIRA
    • Copy the HTML and convert it to markdown with an online converter
    • Add the content to CHANGES.md and update formatting
    • Commit the update to CHANGES.md and make sure it is committed to both release and master branches

1. Run the prepare script

dev/prepare-release.sh <version> <rc-number>

This runs maven’s release prepare with a consistent tag name. After this step, the release tag will exist in the git repository.

If this step fails, you can roll back the changes by running these commands.

find ./ -type f -name '*.releaseBackup' -exec rm {} \;
find ./ -type f -name 'pom.xml' -exec git checkout {} \;

2. Run release:perform to stage binaries

mvn release:perform

This uploads binary artifacts for the release tag to Nexus.

3. In Nexus, close the staging repository

Closing a staging repository makes the binaries available in staging, but does not publish them.

  1. Go to Nexus.
  2. In the menu on the left, choose “Staging Repositories”.
  3. Select the Parquet repository.
  4. At the top, click “Close” and follow the instructions. For the comment use “Apache Parquet [Format] ”.

4. Run the source tarball script

dev/source-release.sh <version> <rc-number>

This script builds the source tarball from the release tag’s SHA1, signs it, and uploads the necessary files with SVN.

The source release is pushed to https://dist.apache.org/repos/dist/dev/parquet/

The last message from the script is the release commit’s SHA1 hash and URL for the VOTE e-mail.

5. Send a VOTE e-mail to dev@parquet.apache.org

Here is a template you can use. Make sure everything applies to your release.

Subject: [VOTE] Release Apache Parquet <VERSION> RC<NUM>


Hi everyone,

I propose the following RC to be released as official Apache Parquet <VERSION> release.

The commit id is <SHA1>
* This corresponds to the tag: apache-parquet-<VERSION>-rc<NUM>
* https://github.com/apache/parquet-mr/tree/<SHA1>

The release tarball, signature, and checksums are here:
* https://dist.apache.org/repos/dist/dev/parquet/<PATH>

You can find the KEYS file here:
* https://downloads.apache.org/parquet/KEYS

Binary artifacts are staged in Nexus here:
* https://repository.apache.org/content/groups/staging/org/apache/parquet/

This release includes important changes that I should have summarized here, but I'm lazy.

Please download, verify, and test.

Please vote in the next 72 hours.

[ ] +1 Release this as Apache Parquet <VERSION>
[ ] +0
[ ] -1 Do not release this because...

Publishing after the vote passes

After a release candidate passes a vote, the candidate needs to be published as the final release.

1. Tag final release and set development version

dev/finalize-release <release-version> <rc-num> <new-development-version-without-SNAPSHOT-suffix>

This will add the final release tag to the RC tag and sets the new development version in the pom files. If everything is fine push the changes and the new tag to github: git push --follow-tags

2. Release the binary repository in Nexus

Releasing a binary repository publishes the binaries to public.

  1. Go to Nexus.
  2. In the menu on the left, choose “Staging Repositories”.
  3. Select the Parquet repository.
  4. At the top, click Release and follow the instructions. For the comment use “Apache Parquet [Format] ”.

3. Copy the release artifacts in SVN into releases

First, check out the candidates and releases locations in SVN:

mkdir parquet
cd parquet
svn co https://dist.apache.org/repos/dist/dev/parquet candidates
svn co https://dist.apache.org/repos/dist/release/parquet releases

Next, copy the directory for the release candidate the passed from candidates to releases and rename it; remove the “-rcN” part of the directory name.

cp -r candidates/apache-parquet-<VERSION>-rcN/ releases/apache-parquet-<VERSION>

Then add and commit the release artifacts:

cd releases
svn add apache-parquet-<version>
svn ci -m "Parquet: Add release <VERSION>"

4. Update parquet.apache.org

Update the downloads page on parquet.apache.org. Instructions for updating the site are on the contribution page.

5. Send an ANNOUNCE e-mail to announce@apache.org and the dev list

[ANNOUNCE] Apache Parquet release <VERSION>


I'm please to announce the release of Parquet <VERSION>!

Parquet is a general-purpose columnar file format for nested data. It uses
space-efficient encodings and a compressed and splittable structure for
processing frameworks like Hadoop.

Changes are listed at: https://github.com/apache/parquet-mr/blob/apache-parquet-<VERSION>/CHANGES.md

This release can be downloaded from: https://parquet.apache.org/downloads/

Java artifacts are available from Maven Central.

Thanks to everyone for contributing!

5 - Resources

Various resources to learn about the Parquet File Format.

5.2 - Presentations

Presentations with content about the Parquet File Format.

5.2.1 - Spark Summit 2020

Spark Summit 2020: The Apache Spark File Format Ecosystem

Slides

5.2.2 - Hadoop Summit 2014

Hadoop Summit 2014: Efficient Data Storage for Analytics with Parquet 2.0

Slides

5.2.3 - #CONF 2014

#CONF 2014: Parquet Format at Twitter

5.2.4 - Strata 2013

Strata 2013: Parquet: Columnar storage for the people

Slides

6 - Apache Software Foundation (ASF)

Apache Software Foundation

6.1 - License

License

6.2 - Security

Security

6.3 - Sponsor

Sponsor

6.4 - Donate

Donate

6.5 - Events

Events