1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.mapreduce.lib.output;
20
21 import java.io.IOException;
22 import java.util.Arrays;
23
24 import org.apache.hadoop.fs.FileSystem;
25 import org.apache.hadoop.fs.Path;
26 import org.apache.hadoop.fs.FileUtil;
27
28 import org.apache.hadoop.io.MapFile;
29 import org.apache.hadoop.io.WritableComparable;
30 import org.apache.hadoop.io.Writable;
31 import org.apache.hadoop.io.SequenceFile.CompressionType;
32 import org.apache.hadoop.io.compress.CompressionCodec;
33 import org.apache.hadoop.io.compress.DefaultCodec;
34 import org.apache.hadoop.mapreduce.Partitioner;
35 import org.apache.hadoop.mapreduce.RecordWriter;
36 import org.apache.hadoop.mapreduce.TaskAttemptContext;
37 import org.apache.hadoop.util.ReflectionUtils;
38 import org.apache.hadoop.classification.InterfaceAudience;
39 import org.apache.hadoop.classification.InterfaceStability;
40 import org.apache.hadoop.conf.Configuration;
41
42
43
44
45
46 @InterfaceAudience.Public
47 @InterfaceStability.Stable
48 public class MapFileOutputFormat
49 extends FileOutputFormat<WritableComparable<?>, Writable> {
50
51 public RecordWriter<WritableComparable<?>, Writable> getRecordWriter(
52 TaskAttemptContext context) throws IOException {
53 Configuration conf = context.getConfiguration();
54 CompressionCodec codec = null;
55 CompressionType compressionType = CompressionType.NONE;
56 if (getCompressOutput(context)) {
57
58 compressionType = SequenceFileOutputFormat.getOutputCompressionType(context);
59
60
61 Class<?> codecClass = getOutputCompressorClass(context,
62 DefaultCodec.class);
63 codec = (CompressionCodec) ReflectionUtils.newInstance(codecClass, conf);
64 }
65
66 Path file = getDefaultWorkFile(context, "");
67 FileSystem fs = file.getFileSystem(conf);
68
69 final MapFile.Writer out =
70 new MapFile.Writer(conf, fs, file.toString(),
71 context.getOutputKeyClass().asSubclass(WritableComparable.class),
72 context.getOutputValueClass().asSubclass(Writable.class),
73 compressionType, codec, context);
74
75 return new RecordWriter<WritableComparable<?>, Writable>() {
76 public void write(WritableComparable<?> key, Writable value)
77 throws IOException {
78 out.append(key, value);
79 }
80
81 public void close(TaskAttemptContext context) throws IOException {
82 out.close();
83 }
84 };
85 }
86
87
88 public static MapFile.Reader[] getReaders(Path dir,
89 Configuration conf) throws IOException {
90 FileSystem fs = dir.getFileSystem(conf);
91 Path[] names = FileUtil.stat2Paths(fs.listStatus(dir));
92
93
94 Arrays.sort(names);
95
96 MapFile.Reader[] parts = new MapFile.Reader[names.length];
97 for (int i = 0; i < names.length; i++) {
98 parts[i] = new MapFile.Reader(fs, names[i].toString(), conf);
99 }
100 return parts;
101 }
102
103
104 public static <K extends WritableComparable<?>, V extends Writable>
105 Writable getEntry(MapFile.Reader[] readers,
106 Partitioner<K, V> partitioner, K key, V value) throws IOException {
107 int part = partitioner.getPartition(key, value, readers.length);
108 return readers[part].get(key, value);
109 }
110 }
111