blob: 7c91a241f5cafbd1e1f616a87ddf505eb3b8ffb0 [file] [log] [blame]
<?xml version="1.0" encoding="UTF-8"?>
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN"
"http://forrest.apache.org/dtd/document-v20.dtd" [
<!ENTITY % avro-entities PUBLIC "-//Apache//ENTITIES Avro//EN"
"../../../../build/avro.ent">
%avro-entities;
]>
<document>
<header>
<title>Apache Avro&#153; &AvroVersion; Hadoop MapReduce guide</title>
</header>
<body>
<p>
Avro provides a convenient way to represent complex data structures within
a Hadoop MapReduce job. Avro data can be used as both input to and output
from a MapReduce job, as well as the intermediate format. The example in
this guide uses Avro data for all three, but it's possible to mix and
match; for instance, MapReduce can be used to aggregate a particular field
in an Avro record.
</p>
<p>
This guide assumes basic familiarity with both Hadoop MapReduce and Avro.
See the <a href="http://hadoop.apache.org/docs/current/">Hadoop
documentation</a> and the <a href="gettingstartedjava.html">Avro getting
started guide</a> for introductions to these projects. This guide uses
the old MapReduce API (<code>org.apache.hadoop.mapred</code>) and the new
MapReduce API (<code>org.apache.hadoop.mapreduce</code>).
</p>
<section>
<title>Setup</title>
<p>
The code from this guide is included in the Avro docs under
<em>examples/mr-example</em>. The example is set up as a Maven project
that includes the necessary Avro and MapReduce dependencies and the Avro
Maven plugin for code generation, so no external jars are needed to run
the example. In particular, the POM includes the following dependencies:
</p>
<source>
&#60;dependency&#62;
&#60;groupId&#62;org.apache.avro&#60;/groupId&#62;
&#60;artifactId&#62;avro&#60;/artifactId&#62;
&#60;version&#62;&AvroVersion;&#60;/version&#62;
&#60;/dependency&#62;
&#60;dependency&#62;
&#60;groupId&#62;org.apache.avro&#60;/groupId&#62;
&#60;artifactId&#62;avro-mapred&#60;/artifactId&#62;
&#60;version&#62;&AvroVersion;&#60;/version&#62;
&#60;/dependency&#62;
&#60;dependency&#62;
&#60;groupId&#62;org.apache.hadoop&#60;/groupId&#62;
&#60;artifactId&#62;hadoop-core&#60;/artifactId&#62;
&#60;version&#62;1.1.0&#60;/version&#62;
&#60;/dependency&#62;
</source>
<p>
And the following plugin:
</p>
<source>
&#60;plugin>
&#60;groupId>org.apache.avro&#60;/groupId>
&#60;artifactId>avro-maven-plugin&#60;/artifactId>
&#60;version>&AvroVersion;&#60;/version>
&#60;executions>
&#60;execution>
&#60;phase>generate-sources&#60;/phase>
&#60;goals>
&#60;goal>schema&#60;/goal>
&#60;/goals>
&#60;configuration>
&#60;sourceDirectory>${project.basedir}/../&#60;/sourceDirectory>
&#60;outputDirectory>${project.basedir}/target/generated-sources/&#60;/outputDirectory>
&#60;/configuration>
&#60;/execution>
&#60;/executions>
&#60;/plugin>
</source>
<p>
If you do not configure the <em>sourceDirectory</em> and <em>outputDirectory</em>
properties, the defaults will be used. The <em>sourceDirectory</em> property
defaults to <em>src/main/avro</em>. The <em>outputDirectory</em> property
defaults to <em>target/generated-sources</em>. You can change the paths to
match your project layout.
</p>
<p>
Alternatively, Avro jars can be downloaded directly from the <a
href="http://avro.apache.org/releases.html">Apache Avro&#153;
Releases</a> page. The relevant Avro jars for this guide are
<em>avro-&AvroVersion;.jar</em> and
<em>avro-mapred-&AvroVersion;.jar</em>, as well as
<em>avro-tools-&AvroVersion;.jar</em> for code generation and viewing
Avro data files as JSON. In addition, you will need to install Hadoop
in order to use MapReduce.
</p>
</section>
<section>
<title>Example: ColorCount</title>
<p>
Below is a simple example of a MapReduce that uses Avro. There is an example
for both the old (<em>org.apache.hadoop.mapred</em>) and new
(<em>org.apache.hadoop.mapreduce</em>) APIs under
<em>examples/mr-example/src/main/java/example/</em>. <em>MapredColorCount</em>
is the example for the older mapred API while <em>MapReduceColorCount</em> is
the example for the newer mapreduce API. Both examples are below, but
we will detail the mapred API in our subsequent examples.
</p>
<p>MapredColorCount:</p>
<source>
package example;
import java.io.IOException;
import org.apache.avro.*;
import org.apache.avro.Schema.Type;
import org.apache.avro.mapred.*;
import org.apache.hadoop.conf.*;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.*;
import org.apache.hadoop.util.*;
import example.avro.User;
public class MapredColorCount extends Configured implements Tool {
public static class ColorCountMapper extends AvroMapper&#60;User, Pair&#60;CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector&#60;Pair&#60;CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair&#60;CharSequence, Integer>(color, 1));
}
}
public static class ColorCountReducer extends AvroReducer&#60;CharSequence, Integer,
Pair&#60;CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable&#60;Integer> values,
AvroCollector&#60;Pair&#60;CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair&#60;CharSequence, Integer>(key, sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println("Usage: MapredColorCount &#60;input path> &#60;output path>");
return -1;
}
JobConf conf = new JobConf(getConf(), MapredColorCount.class);
conf.setJobName("colorcount");
FileInputFormat.setInputPaths(conf, new Path(args[0]));
FileOutputFormat.setOutputPath(conf, new Path(args[1]));
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setReducerClass(conf, ColorCountReducer.class);
// Note that AvroJob.setInputSchema and AvroJob.setOutputSchema set
// relevant config options such as input/output format, map output
// classes, and output key class.
AvroJob.setInputSchema(conf, User.getClassSchema());
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
JobClient.runJob(conf);
return 0;
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new Configuration(), new MapredColorCount(), args);
System.exit(res);
}
}
</source>
<p>MapReduceColorCount:</p>
<source>
package example;
import java.io.IOException;
import org.apache.avro.Schema;
import org.apache.avro.mapred.AvroKey;
import org.apache.avro.mapred.AvroValue;
import org.apache.avro.mapreduce.AvroJob;
import org.apache.avro.mapreduce.AvroKeyInputFormat;
import org.apache.avro.mapreduce.AvroKeyValueOutputFormat;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.Reducer;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import example.avro.User;
public class MapReduceColorCount extends Configured implements Tool {
public static class ColorCountMapper extends
Mapper&lt;AvroKey&lt;User&gt;, NullWritable, Text, IntWritable&gt; {
@Override
public void map(AvroKey&lt;User&gt; key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = &quot;none&quot;;
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
public static class ColorCountReducer extends
Reducer&lt;Text, IntWritable, AvroKey&lt;CharSequence&gt;, AvroValue&lt;Integer&gt;&gt; {
@Override
public void reduce(Text key, Iterable&lt;IntWritable&gt; values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey&lt;CharSequence&gt;(key.toString()), new AvroValue&lt;Integer&gt;(sum));
}
}
public int run(String[] args) throws Exception {
if (args.length != 2) {
System.err.println(&quot;Usage: MapReduceColorCount &lt;input path&gt; &lt;output path&gt;&quot;);
return -1;
}
Job job = new Job(getConf());
job.setJarByClass(MapReduceColorCount.class);
job.setJobName(&quot;Color Count&quot;);
FileInputFormat.setInputPaths(job, new Path(args[0]));
FileOutputFormat.setOutputPath(job, new Path(args[1]));
job.setInputFormatClass(AvroKeyInputFormat.class);
job.setMapperClass(ColorCountMapper.class);
AvroJob.setInputKeySchema(job, User.getClassSchema());
job.setMapOutputKeyClass(Text.class);
job.setMapOutputValueClass(IntWritable.class);
job.setOutputFormatClass(AvroKeyValueOutputFormat.class);
job.setReducerClass(ColorCountReducer.class);
AvroJob.setOutputKeySchema(job, Schema.create(Schema.Type.STRING));
AvroJob.setOutputValueSchema(job, Schema.create(Schema.Type.INT));
return (job.waitForCompletion(true) ? 0 : 1);
}
public static void main(String[] args) throws Exception {
int res = ToolRunner.run(new MapReduceColorCount(), args);
System.exit(res);
}
}
</source>
<p>
ColorCount reads in data files containing <code>User</code> records,
defined in <em>examples/user.avsc</em>, and counts the number of
instances of each favorite color. (This example draws inspiration from
the canonical WordCount MapReduce application.) This example uses the
old MapReduce API. See MapReduceAvroWordCount, found under
<em>doc/examples/mr-example/src/main/java/example/</em> to see the new MapReduce
API example. The <code>User</code>
schema is defined as follows:
</p>
<source>
{"namespace": "example.avro",
"type": "record",
"name": "User",
"fields": [
{"name": "name", "type": "string"},
{"name": "favorite_number", "type": ["int", "null"]},
{"name": "favorite_color", "type": ["string", "null"]}
]
}
</source>
<p>
This schema is compiled into the <code>User</code> class used by
ColorCount via the Avro Maven plugin (see
<em>examples/mr-example/pom.xml</em> for how this is set up).
</p>
<p>
ColorCountMapper essentially takes a <code>User</code> as input and
extracts the <code>User</code>'s favorite color, emitting the key-value
pair <code>&#60;</code><em>favoriteColor</em><code>, 1></code>.
ColorCountReducer then adds up how many occurrences of a particular
favorite color were emitted, and outputs the result as a
<code>Pair</code> record. These <code>Pair</code>s are serialized to an
Avro data file.
</p>
<section>
<title>Running ColorCount</title>
<p>
The ColorCount application is provided as a Maven project in the Avro
docs under <em>examples/mr-example</em>. To build the project,
including the code generation of the User schema, run:
</p>
<source>
mvn compile
</source>
<p>
Next, run GenerateData from examples/mr-examples to create an Avro data
file, <em>input/users.avro</em>, containing 20 <code>User</code>s with
favorite colors chosen randomly from a list:
</p>
<source>
mvn exec:java -q -Dexec.mainClass=example.GenerateData
</source>
<p>
Besides creating the data file, GenerateData prints the JSON
representations of the Users generated to stdout, for example:
</p>
<source>
{"name": "user", "favorite_number": null, "favorite_color": "red"}
{"name": "user", "favorite_number": null, "favorite_color": "green"}
{"name": "user", "favorite_number": null, "favorite_color": "purple"}
{"name": "user", "favorite_number": null, "favorite_color": null}
...
</source>
<p>
Now we're ready to run ColorCount. We specify our freshly-generated
<em>input</em> folder as the input path and <em>output</em> as our
output folder (note that MapReduce will not start a job if the output
folder already exists):
</p>
<source>
mvn exec:java -q -Dexec.mainClass=example.MapredColorCount -Dexec.args="input output"
</source>
<p>
Once ColorCount completes, checking the contents of the new
<em>output</em> directory should yield the following:
</p>
<source>
$ ls output/
part-00000.avro _SUCCESS
</source>
<p>
You can check the contents of the generated Avro file using the avro-tools jar:
</p>
<source>
$ java -jar /path/to/avro-tools-&AvroVersion;.jar tojson output/part-00000.avro
{"value": 3, "key": "blue"}
{"value": 7, "key": "green"}
{"value": 1, "key": "none"}
{"value": 2, "key": "orange"}
{"value": 3, "key": "purple"}
{"value": 2, "key": "red"}
{"value": 2, "key": "yellow"}
</source>
</section>
</section>
<p>Now let's go over the ColorCount example in detail.</p>
<section>
<title>Mapper - org.apache.hadoop.mapred API</title>
<p>
The easiest way to use Avro data files as input to a MapReduce job is to
subclass <code>AvroMapper</code>. An <code>AvroMapper</code> defines a
map function that takes an Avro datum as input and outputs a key/value
pair represented as a <code>Pair</code> record. In the ColorCount
example, <code>ColorCountMapper</code> is an <code>AvroMapper</code>
that takes a <code>User</code> as input and outputs a
<code>Pair&#60;CharSequence, Integer>></code>, where the
<code>CharSequence</code> key is the user's favorite color and the
<code>Integer</code> value is 1.
</p>
<source>
public static class ColorCountMapper extends AvroMapper&#60;User, Pair&#60;CharSequence, Integer>> {
@Override
public void map(User user, AvroCollector&#60;Pair&#60;CharSequence, Integer>> collector, Reporter reporter)
throws IOException {
CharSequence color = user.getFavoriteColor();
// We need this check because the User.favorite_color field has type ["string", "null"]
if (color == null) {
color = "none";
}
collector.collect(new Pair&#60;CharSequence, Integer>(color, 1));
}
}
</source>
<p>
In order to use our <code>AvroMapper</code>, we must call
<code>AvroJob.setMapperClass</code> and
<code>AvroJob.setInputSchema</code>.
</p>
<source>
AvroJob.setMapperClass(conf, ColorCountMapper.class);
AvroJob.setInputSchema(conf, User.getClassSchema());
</source>
<p>
Note that <code>AvroMapper</code> does not implement the
<code>Mapper</code> interface. Under the hood, the specified Avro data
files are deserialized into <code>AvroWrapper</code>s containing the
actual data, which are processed by a <code>Mapper</code> that calls the
configured <code>AvroMapper</code>'s map function.
<code>AvroJob.setInputSchema</code> sets up the relevant configuration
parameters needed to make this happen, thus you should not need to call
<code>JobConf.setMapperClass</code>,
<code>JobConf.setInputFormat</code>,
<code>JobConf.setMapOutputKeyClass</code>,
<code>JobConf.setMapOutputValueClass</code>, or
<code>JobConf.setOutputKeyComparatorClass</code>.
</p>
</section>
<section>
<title>Mapper - org.apache.hadoop.mapreduce API</title>
<p>
This document will not go into all the differences between the mapred and mapreduce APIs,
however will describe the main differences. As you can see, ColorCountMapper is now a
subclass of the Hadoop Mapper class and is passed an AvroKey as it's key.
Additionally, the AvroJob method calls were slightly changed.
</p>
<source>
public static class ColorCountMapper extends
Mapper&lt;AvroKey&lt;User&gt;, NullWritable, Text, IntWritable&gt; {
@Override
public void map(AvroKey&lt;User&gt; key, NullWritable value, Context context)
throws IOException, InterruptedException {
CharSequence color = key.datum().getFavoriteColor();
if (color == null) {
color = &quot;none&quot;;
}
context.write(new Text(color.toString()), new IntWritable(1));
}
}
</source>
</section>
<section>
<title>Reducer - org.apache.hadoop.mapred API</title>
<p>
Analogously to <code>AvroMapper</code>, an <code>AvroReducer</code>
defines a reducer function that takes the key/value types output by an
<code>AvroMapper</code> (or any mapper that outputs <code>Pair</code>s)
and outputs a key/value pair represented a <code>Pair</code> record. In
the ColorCount example, <code>ColorCountReducer</code> is an
<code>AvroReducer</code> that takes the <code>CharSequence</code> key
representing a favorite color and the <code>Iterable&#60;Integer></code>
representing the counts for that color (they should all be 1 in this
example) and adds up the counts.
</p>
<source>
public static class ColorCountReducer extends AvroReducer&#60;CharSequence, Integer,
Pair&#60;CharSequence, Integer>> {
@Override
public void reduce(CharSequence key, Iterable&#60;Integer> values,
AvroCollector&#60;Pair&#60;CharSequence, Integer>> collector,
Reporter reporter)
throws IOException {
int sum = 0;
for (Integer value : values) {
sum += value;
}
collector.collect(new Pair&#60;CharSequence, Integer>(key, sum));
}
}
</source>
<p>
In order to use our <code>AvroReducer</code>, we must call
<code>AvroJob.setReducerClass</code> and
<code>AvroJob.setOutputSchema</code>.
</p>
<source>
AvroJob.setReducerClass(conf, ColorCountReducer.class);
AvroJob.setOutputSchema(conf, Pair.getPairSchema(Schema.create(Type.STRING),
Schema.create(Type.INT)));
</source>
<p>
Note that <code>AvroReducer</code> does not implement the
<code>Reducer</code> interface. The intermediate <code>Pair</code>s
output by the mapper are split into <code>AvroKey</code>s and
<code>AvroValue</code>s, which are processed by a <code>Reducer</code>
that calls the configured <code>AvroReducer</code>'s reduce function.
<code>AvroJob.setOutputSchema</code> sets up the relevant configuration
parameters needed to make this happen, thus you should not need to call
<code>JobConf.setReducerClass</code>,
<code>JobConf.setOutputFormat</code>,
<code>JobConf.setOutputKeyClass</code>,
<code>JobConf.setMapOutputKeyClass</code>,
<code>JobConf.setMapOutputValueClass</code>, or
<code>JobConf.setOutputKeyComparatorClass</code>.
</p>
</section>
<section>
<title>Reduce - org.apache.hadoop.mapreduce API</title>
<p>
As before we not detail every difference between the APIs. As with the Mapper
change ColorCountReducer is now a subclass of Reducer and AvroKey and AvroValue
are emitted.
Additionally, the AvroJob method calls were slightly changed.
</p>
<source>
public static class ColorCountReducer extends
Reducer&lt;Text, IntWritable, AvroKey&lt;CharSequence&gt;, AvroValue&lt;Integer&gt;&gt; {
@Override
public void reduce(Text key, Iterable&lt;IntWritable&gt; values,
Context context) throws IOException, InterruptedException {
int sum = 0;
for (IntWritable value : values) {
sum += value.get();
}
context.write(new AvroKey&lt;CharSequence&gt;(key.toString()), new AvroValue&lt;Integer&gt;(sum));
}
}
</source>
</section>
<section>
<title>Learning more</title>
<p>
The mapred API allows users to mix Avro <code>AvroMapper</code>s and
<code>AvroReducer</code>s with non-Avro <code>Mapper</code>s and
<code>Reducer</code>s and the mapreduce API allows users input Avro
and output non-Avro or vice versa.
</p>
<p>
The mapred package has API <a
href="http://avro.apache.org/docs/current/api/java/org/apache/avro/mapred/package-summary.html">
<code>org.apache.avro.mapred</code> documentation</a> as does the <a
href="http://avro.apache.org/docs/current/api/java/org/apache/avro/mapreduce/package-summary.html">
<code>org.apache.avro.mapreduce</code> package</a>.
MapReduce API (<code>org.apache.hadoop.mapreduce</code>). Similarily to the mapreduce package,
it's possible with the mapred API to implement your own <code>Mapper</code>s and
<code>Reducer</code>s directly using the public classes provided in
these libraries. See the AvroWordCount application, found under
<em>examples/mr-example/src/main/java/example/AvroWordCount.java</em> in
the Avro documentation, for an example of implementing a
<code>Reducer</code> that outputs Avro data using the old MapReduce API.
See the MapReduceAvroWordCount application, found under
<em>examples/mr-example/src/main/java/example/MapReduceAvroWordCount.java</em> in
the Avro documentation, for an example of implementing a
<code>Reducer</code> that outputs Avro data using the new MapReduce API.
</p>
</section>
</body>
</document>