Parquet-MR contains the java implementation of the Parquet format. Parquet is a columnar storage format for Hadoop; it provides efficient storage and encoding of data. Parquet uses the record shredding and assembly algorithm described in the Dremel paper to represent nested structures.
You can find some details about the format and intended use cases in our Hadoop Summit 2013 presentation
Parquet is a very active project, and new features are being added quickly; below is the state as of June 2013.
Feature | In trunk | In dev | Planned | Expected release |
---|---|---|---|---|
Type-specific encoding | YES | 1.0 | ||
Hive integration | YES (28) | 1.0 | ||
Pig integration | YES | 1.0 | ||
Cascading integration | YES | 1.0 | ||
Crunch integration | YES (CRUNCH-277) | 1.0 | ||
Impala integration | YES (non-nested) | 1.0 | ||
Java Map/Reduce API | YES | 1.0 | ||
Native Avro support | YES | 1.0 | ||
Native Thrift support | YES | 1.0 | ||
Complex structure support | YES | 1.0 | ||
Future-proofed versioning | YES | 1.0 | ||
RLE | YES | 1.0 | ||
Bit Packing | YES | 1.0 | ||
Adaptive dictionary encoding | YES | 1.0 | ||
Predicate pushdown | YES (68) | 1.0 | ||
Column stats | YES | 2.0 | ||
Delta encoding | YES | 2.0 | ||
Native Protocol Buffers support | YES | 2.0 | ||
Index pages | YES | 2.0 |
Input and Output formats. Note that to use an Input or Output format, you need to implement a WriteSupport or ReadSupport class, which will implement the conversion of your object to and from a Parquet schema.
We've implemented this for 2 popular data formats to provide a clean migration path as well:
Thrift integration is provided by the parquet-thrift sub-project. If you are using Thrift through Scala, you may be using Twitter's Scrooge. If that's the case, not to worry -- we took care of the Scrooge/Apache Thrift glue for you in the parquet-scrooge sub-project.
Avro conversion is implemented via the parquet-avro sub-project.
- The ParquetOutputFormat can be provided a WriteSupport to write your own objects to an event based RecordConsumer.
- the ParquetInputFormat can be provided a ReadSupport to materialize your own objects by implementing a RecordMaterializer
See the APIs:
A Loader and a Storer are provided to read and write Parquet files with Apache Pig
Storing data into Parquet in Pig is simple:
-- options you might want to fiddle with
SET parquet.page.size 1048576 -- default. this is your min read/write unit.
SET parquet.block.size 134217728 -- default. your memory budget for buffering data
SET parquet.compression lzo -- or you can use none, gzip, snappy
STORE mydata into '/some/path' USING parquet.pig.ParquetStorer;
Reading in Pig is also simple:
mydata = LOAD '/some/path' USING parquet.pig.ParquetLoader();
If the data was stored using Pig, things will "just work". If the data was stored using another method, you will need to provide the Pig schema equivalent to the data you stored (you can also write the schema to the file footer while writing it -- but that's pretty advanced). We will provide a basic automatic schema conversion soon.
Hive integration is provided via the parquet-hive sub-project.
to run the unit tests: mvn test
to build the jars: mvn package
The build runs in Travis CI:
- apis documentation
- maven dependency:
<repositories>
<repository>
<id>sonatype-nexus-snapshots</id>
<url>https://oss.sonatype.org/content/repositories/snapshots</url>
<releases>
<enabled>false</enabled>
</releases>
<snapshots>
<enabled>true</enabled>
</snapshots>
</repository>
</repositories>
<dependencies>
<dependency>
<groupId>com.twitter</groupId>
<artifactId>parquet-common</artifactId>
<version>1.0.0-SNAPSHOT</version>
</dependency>
<dependency>
<groupId>com.twitter</groupId>
<artifactId>parquet-encoding</artifactId>
<version>1.0.0-SNAPSHOT</version>
</dependency>
<dependency>
<groupId>com.twitter</groupId>
<artifactId>parquet-column</artifactId>
<version>1.0.0-SNAPSHOT</version>
</dependency>
<dependency>
<groupId>com.twitter</groupId>
<artifactId>parquet-hadoop</artifactId>
<version>1.0.0-SNAPSHOT</version>
</dependency>
</dependencies>
- apis documentation
- maven dependency:
<dependencies>
<dependency>
<groupId>com.twitter</groupId>
<artifactId>parquet-common</artifactId>
<version>1.0.0</version>
</dependency>
<dependency>
<groupId>com.twitter</groupId>
<artifactId>parquet-encoding</artifactId>
<version>1.0.0</version>
</dependency>
<dependency>
<groupId>com.twitter</groupId>
<artifactId>parquet-column</artifactId>
<version>1.0.0</version>
</dependency>
<dependency>
<groupId>com.twitter</groupId>
<artifactId>parquet-hadoop</artifactId>
<version>1.0.0</version>
</dependency>
</dependencies>
If you are looking for some ideas on what to contribute, check out GitHub issues for this project labeled "Pick me up!". Comment on the issue and/or contact the parquet-dev group with your questions and ideas.
We tend to do fairly close readings of pull requests, and you may get a lot of comments. Some common issues that are not code structure related, but still important:
- Please make sure to add the license headers to all new files. You can do this automatically by using the
mvn license:format
command. - Use 2 spaces for whitespace. Not tabs, not 4 spaces. The number of the spacing shall be 2.
- Give your operators some room. Not
a+b
buta + b
and notfoo(int a,int b)
butfoo(int a, int b)
. - Generally speaking, stick to the Sun Java Code Conventions
- Make sure tests pass!
- Julien Le Dem @J_ https://github.com/julienledem
- Tom White https://github.com/tomwhite
- Mickaël Lacour https://github.com/mickaellcr
- Remy Pecqueur https://github.com/Lordshinjo
- Avi Bryant https://github.com/avibryant
- Dmitriy Ryaboy @squarecog https://github.com/dvryaboy
- Jonathan Coveney http://twitter.com/jco
- and many others -- see the Contributor report
- google group https://groups.google.com/d/forum/parquet-dev
- the group email address: [email protected]
Copyright 2012-2013 Twitter, Inc.
Licensed under the Apache License, Version 2.0: http://www.apache.org/licenses/LICENSE-2.0