forked from twitter/elephant-bird
/
ProtobufMRExample.java
182 lines (148 loc) · 6.91 KB
/
ProtobufMRExample.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
package com.twitter.elephantbird.examples;
import java.io.IOException;
import java.util.StringTokenizer;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.LongWritable;
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.input.TextInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
import org.apache.hadoop.util.GenericOptionsParser;
import com.twitter.elephantbird.examples.proto.Examples.Age;
import com.twitter.elephantbird.mapreduce.input.MultiInputFormat;
import com.twitter.elephantbird.mapreduce.io.ProtobufWritable;
import com.twitter.elephantbird.mapreduce.output.LzoProtobufB64LineOutputFormat;
import com.twitter.elephantbird.mapreduce.output.LzoProtobufBlockOutputFormat;
/**
* -Dproto.test=lzoOut : takes text files with name and age on each line as
* input and writes to lzo file with Protobuf serilized data. <br>
* -Dproto.test=lzoIn : does the reverse. <br><br>
*
* -Dproto.test.format=Block (or B64Line) to test different formats. <br>
*/
public class ProtobufMRExample {
// This is intentionally very similar to ThriftMRExample.
private ProtobufMRExample() {}
public static class TextMapper extends Mapper<LongWritable, Text, NullWritable, ProtobufWritable<Age>> {
ProtobufWritable<Age> protoWritable = ProtobufWritable.newInstance(Age.class);
@Override
protected void map(LongWritable key, Text value, Context context) throws IOException, InterruptedException {
StringTokenizer line = new StringTokenizer(value.toString(), "\t\r\n");
String name;
if (line.hasMoreTokens()
&& (name = line.nextToken()) != null
&& line.hasMoreTokens()) {
protoWritable.set(Age.newBuilder()
.setName(name)
.setAge(Integer.parseInt(line.nextToken()))
.build());
context.write(null, protoWritable);
}
}
}
public int runTextToLzo(String[] args, Configuration conf) throws Exception {
Job job = new Job(conf);
job.setJobName("Protobuf Example : Text to LzoB64Line");
job.setJarByClass(getClass());
job.setMapperClass(TextMapper.class);
job.setNumReduceTasks(0);
job.setInputFormatClass(TextInputFormat.class);
if (conf.get("proto.test.format", "B64Line").equals("Block")) {
LzoProtobufBlockOutputFormat.setClassConf(Age.class, job.getConfiguration());
job.setOutputFormatClass(LzoProtobufBlockOutputFormat.class);
} else { // assume B64Line
LzoProtobufB64LineOutputFormat.setClassConf(Age.class, job.getConfiguration());
job.setOutputFormatClass(LzoProtobufB64LineOutputFormat.class);
}
FileInputFormat.setInputPaths(job, new Path(args[0]));
FileOutputFormat.setOutputPath(job, new Path(args[1]));
return job.waitForCompletion(true) ? 0 : 1;
}
public static class LzoMapper extends Mapper<LongWritable, ProtobufWritable<Age>, Text, Text> {
@Override
protected void map(LongWritable key, ProtobufWritable<Age> value, Context context) throws IOException, InterruptedException {
Age age = value.get();
context.write(null, new Text(age.getName() + "\t" + age.getAge()));
}
}
int runLzoToText(String[] args, Configuration conf) throws Exception {
Job job = new Job(conf);
job.setJobName("Protobuf Example : LzoB64Line to Text");
job.setJarByClass(getClass());
job.setMapperClass(LzoMapper.class);
job.setNumReduceTasks(0);
// input format is same for both B64Line or block:
MultiInputFormat.setInputFormatClass(Age.class, job);
job.setOutputFormatClass(TextOutputFormat.class);
FileInputFormat.setInputPaths(job, new Path(args[0]));
FileOutputFormat.setOutputPath(job, new Path(args[1]));
return job.waitForCompletion(true) ? 0 : 1;
}
public static class SortMapper extends Mapper<LongWritable, Text, Text, ProtobufWritable<Age>> {
ProtobufWritable<Age> protoWritable = ProtobufWritable.newInstance(Age.class);
@Override
protected void map(LongWritable key, Text value, Context context) throws IOException, InterruptedException {
StringTokenizer line = new StringTokenizer(value.toString(), "\t\r\n");
String name;
if (line.hasMoreTokens()
&& (name = line.nextToken()) != null
&& line.hasMoreTokens()) {
protoWritable.set(Age.newBuilder()
.setName(name)
.setAge(Integer.parseInt(line.nextToken()))
.build());
context.write(new Text(name), protoWritable);
}
}
}
public static class SortReducer extends Reducer<Text, ProtobufWritable<Age>, Text, Text> {
@Override
protected void reduce(Text key, Iterable<ProtobufWritable<Age>> values, Context context) throws IOException, InterruptedException {
for(ProtobufWritable<Age> value : values) {
/* setConverter() before get() is required since 'value' object was
* created by MR with default ProtobufWritable's default constructor,
* as result object does not know its runtime Protobuf class.
*/
value.setConverter(Age.class);
context.write(null, new Text(value.get().getName() + "\t" + value.get().getAge()));
}
}
}
int runSorter(String[] args, Configuration conf) throws Exception {
Job job = new Job(conf);
job.setJobName("Protobuf Example : ProtobufWritable as Map output class");
job.setJarByClass(getClass());
job.setMapperClass(SortMapper.class);
job.setMapOutputKeyClass(Text.class);
job.setMapOutputValueClass(ProtobufWritable.class);
job.setReducerClass(SortReducer.class);
job.setNumReduceTasks(1);
job.setInputFormatClass(TextInputFormat.class);
job.setOutputFormatClass(TextOutputFormat.class);
FileInputFormat.setInputPaths(job, new Path(args[0]));
FileOutputFormat.setOutputPath(job, new Path(args[1]));
return job.waitForCompletion(true) ? 0 : 1;
}
public static void main(String[] args) throws Exception {
Configuration conf = new Configuration();
args = new GenericOptionsParser(conf, args).getRemainingArgs();
ProtobufMRExample runner = new ProtobufMRExample();
if (args.length != 2) {
System.out.println("Usage: hadoop jar path/to/this.jar " + runner.getClass() + " <input dir> <output dir>");
System.exit(1);
}
String test = conf.get("proto.test", "lzoIn");
if (test.equals("lzoIn"))
System.exit(runner.runLzoToText(args, conf));
if (test.equals("lzoOut"))
System.exit(runner.runTextToLzo(args, conf));
if (test.equals("sort"))
System.exit(runner.runSorter(args, conf));
}
}