nielsbasjes / splittablegzip

Splittable Gzip codec for Hadoop
Apache License 2.0
69 stars 8 forks source link
codec gzip gzip-codec gzipped-files hadoop mapreduce-java pig spark splittable

Making gzip splittable for Hadoop

Github actions Build status License Maven Central If this project has business value for you then don't hesitate to support me with a small donation. If this project has business value for you then don't hesitate to support me with a small donation.

In many Hadoop production environments you get gzipped files as the raw input. Usually these are Apache HTTPD logfiles. When putting these gzipped files into Hadoop you are stuck with exactly 1 map task per input file. In many scenarios this is fine.

However when doing a lot of work in this very first map task it may very well be advantageous to dividing the work over multiple tasks, even if there is a penalty for this scaling out.

This addon for Hadoop makes this possible.

Benchmark

I did benchmarking jobs to see how this solution scales and performs. The software and the results can be found in the Benchmark folder.

In general we can say that you win as long as there is unused capacity in your cluster.

Graph of the results

Requirements

First of all this only works with Hadoop 1.1.0 and up because this depends on the presence of the SplittableCompressionCodec interface.

I tested it with Hortonworks 2.1.2 and Cloudera CDH 4.5.0.

Downloads

Sources

Currently it can only be downloaded via github.

https://github.com/nielsbasjes/splittablegzip

Binary

For normal projects you can simply download the prebuilt version from maven central. So when using maven you can simply add this to your project

<dependency>
  <groupId>nl.basjes.hadoop</groupId>
  <artifactId>splittablegzip</artifactId>
  <version>1.3</version>
</dependency>

Building

On CDH the build fails if the native gzip could not be loaded for running the unit tests. To fix this you need to install the native package and set the environment so it can find them for your platform. I.e. Do something like this before starting the build or loading your IDE.

export LD_LIBRARY_PATH=/usr/lib/hadoop-0.20/lib/native/Linux-amd64-64

Installation

  1. Place the jar file in the classpath of your hadoop installation.

  2. Enable this codec and make sure the regular GzipCodec is NOT used. This can be done by changing the io.compression.codecs property to something like this:

    org.apache.hadoop.io.compress.DefaultCodec, nl.basjes.hadoop.io.compress.SplittableGzipCodec, org.apache.hadoop.io.compress.BZip2Codec

  3. Set the split size to something that works in your situation. This can be done by setting the appropriate values for mapreduce.input.fileinputformat.split.minsize and/or mapreduce.input.fileinputformat.split.maxsize.

Usage examples

Choosing the configuration settings

How it works

For each "split" the gzipped input file is read from the beginning of the file till the point where the split starts, thus reading, decompressing and discarding (wasting!) everything that is before the start of the split.

FACT: Files compressed with the Gzip codec are NOT SPLITTABLE. Never have been, never will be.

This codec offers a trade off between "spent resources" and "scalability" when reading Gzipped input files by simply always starting at the beginning of the file.

So in general this "splittable" Gzip codec will WASTE CPU time and FileSystem IO (HDFS) and probably other system resources (Network) too to reduce the "wall clock" time in some real-life situations.

When is this useful?

Assume you have a heavy map phase for which the input is a 1GiB Apache httpd logfile. Now assume this map takes 60 minutes of CPU time to run. Then this task will take 60 minutes to run because all of that CPU time must be spent on a single CPU core ... Gzip is not splittable!

This codec will waste CPU power by always starting from the start of the gzipped file and discard all the decompressed data until the start of the split has been reached.

Decompressing a 1GiB Gzip file usually takes only a few (2-4) minutes.

So if a "60 minutes" input file is split into 4 equal parts then:

  1. the 1st map task will
    • process the 1st split (15 minutes)
  2. the 2nd map task will
    • discard the 1st split ( 1 minute ).
    • process the 2nd split (15 minutes).
  3. the 3rd map task will
    • discard the 1st split ( 1 minute ).
    • discard the 2nd split ( 1 minute ).
    • process the 3rd split (15 minutes).
  4. the 4th task will
    • discard the 1st split ( 1 minute ).
    • discard the 2nd split ( 1 minute ).
    • discard the 3rd split ( 1 minute ).
    • process the 4th split (15 minutes).

Because all tasks run in parallel the running time in this example would be 18 minutes (i.e. the worst split time) instead of the normal 60 minutes. We have wasted about 6 minutes of CPU time and completed the job in about 30% of the original wall clock time.

Tuning for optimal performance and scalability.

The overall advise is to EXPERIMENT with the settings and do benchmarks.

Remember that:

A possible optimum:

Alternative approaches

Always remember that there are alternative approaches:

Implementation notes

There were two major hurdles that needed to be solved to make this work:

History

Originally this feature was submitted to be part of the core of Hadoop.

See: Gzip splittable (HADOOP-7076).

Created by

This idea was conceived and implemented by Niels Basjes.