1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.mapreduce;
21
22 import java.io.IOException;
23 import java.net.URI;
24 import java.net.URISyntaxException;
25 import java.util.ArrayList;
26 import java.util.List;
27 import java.util.Map;
28 import java.util.TreeMap;
29 import java.util.TreeSet;
30
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.filecache.DistributedCache;
33 import org.apache.hadoop.fs.FileSystem;
34 import org.apache.hadoop.fs.Path;
35 import org.apache.hadoop.hbase.HConstants;
36 import org.apache.hadoop.hbase.KeyValue;
37 import org.apache.hadoop.hbase.client.HTable;
38 import org.apache.hadoop.hbase.client.Put;
39 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
40 import org.apache.hadoop.hbase.io.hfile.Compression;
41 import org.apache.hadoop.hbase.io.hfile.HFile;
42 import org.apache.hadoop.hbase.mapreduce.hadoopbackport.TotalOrderPartitioner;
43 import org.apache.hadoop.hbase.regionserver.StoreFile;
44 import org.apache.hadoop.hbase.util.Bytes;
45 import org.apache.hadoop.io.NullWritable;
46 import org.apache.hadoop.io.SequenceFile;
47 import org.apache.hadoop.mapreduce.Job;
48 import org.apache.hadoop.mapreduce.RecordWriter;
49 import org.apache.hadoop.mapreduce.TaskAttemptContext;
50 import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
51 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
52
53 import org.apache.commons.logging.Log;
54 import org.apache.commons.logging.LogFactory;
55
56 import com.google.common.base.Preconditions;
57
58
59
60
61
62
63
64
65
66 public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable, KeyValue> {
67 static Log LOG = LogFactory.getLog(HFileOutputFormat.class);
68
69 public RecordWriter<ImmutableBytesWritable, KeyValue> getRecordWriter(final TaskAttemptContext context)
70 throws IOException, InterruptedException {
71
72 final Path outputPath = FileOutputFormat.getOutputPath(context);
73 final Path outputdir = new FileOutputCommitter(outputPath, context).getWorkPath();
74 Configuration conf = context.getConfiguration();
75 final FileSystem fs = outputdir.getFileSystem(conf);
76
77 final long maxsize = conf.getLong("hbase.hregion.max.filesize", 268435456);
78 final int blocksize = conf.getInt("hfile.min.blocksize.size", 65536);
79
80 final String compression = conf.get("hfile.compression",
81 Compression.Algorithm.NONE.getName());
82
83 return new RecordWriter<ImmutableBytesWritable, KeyValue>() {
84
85 private final Map<byte [], WriterLength> writers =
86 new TreeMap<byte [], WriterLength>(Bytes.BYTES_COMPARATOR);
87 private byte [] previousRow = HConstants.EMPTY_BYTE_ARRAY;
88 private final byte [] now = Bytes.toBytes(System.currentTimeMillis());
89
90 public void write(ImmutableBytesWritable row, KeyValue kv)
91 throws IOException {
92 long length = kv.getLength();
93 byte [] family = kv.getFamily();
94 WriterLength wl = this.writers.get(family);
95 if (wl == null || ((length + wl.written) >= maxsize) &&
96 Bytes.compareTo(this.previousRow, 0, this.previousRow.length,
97 kv.getBuffer(), kv.getRowOffset(), kv.getRowLength()) != 0) {
98
99 Path basedir = new Path(outputdir, Bytes.toString(family));
100 if (wl == null) {
101 wl = new WriterLength();
102 this.writers.put(family, wl);
103 if (this.writers.size() > 1) throw new IOException("One family only");
104
105 if (!fs.exists(basedir)) fs.mkdirs(basedir);
106 }
107 wl.writer = getNewWriter(wl.writer, basedir);
108 LOG.info("Writer=" + wl.writer.getPath() +
109 ((wl.written == 0)? "": ", wrote=" + wl.written));
110 wl.written = 0;
111 }
112 kv.updateLatestStamp(this.now);
113 wl.writer.append(kv);
114 wl.written += length;
115
116 this.previousRow = kv.getRow();
117 }
118
119
120
121
122
123
124
125 private HFile.Writer getNewWriter(final HFile.Writer writer,
126 final Path familydir)
127 throws IOException {
128 close(writer);
129 return new HFile.Writer(fs, StoreFile.getUniqueFile(fs, familydir),
130 blocksize, compression, KeyValue.KEY_COMPARATOR);
131 }
132
133 private void close(final HFile.Writer w) throws IOException {
134 if (w != null) {
135 w.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY,
136 Bytes.toBytes(System.currentTimeMillis()));
137 w.appendFileInfo(StoreFile.BULKLOAD_TASK_KEY,
138 Bytes.toBytes(context.getTaskAttemptID().toString()));
139 w.appendFileInfo(StoreFile.MAJOR_COMPACTION_KEY,
140 Bytes.toBytes(true));
141 w.close();
142 }
143 }
144
145 public void close(TaskAttemptContext c)
146 throws IOException, InterruptedException {
147 for (Map.Entry<byte [], WriterLength> e: this.writers.entrySet()) {
148 close(e.getValue().writer);
149 }
150 }
151 };
152 }
153
154
155
156
157 static class WriterLength {
158 long written = 0;
159 HFile.Writer writer = null;
160 }
161
162
163
164
165
166 private static List<ImmutableBytesWritable> getRegionStartKeys(HTable table)
167 throws IOException {
168 byte[][] byteKeys = table.getStartKeys();
169 ArrayList<ImmutableBytesWritable> ret =
170 new ArrayList<ImmutableBytesWritable>(byteKeys.length);
171 for (byte[] byteKey : byteKeys) {
172 ret.add(new ImmutableBytesWritable(byteKey));
173 }
174 return ret;
175 }
176
177
178
179
180
181
182
183 private static void writePartitions(Configuration conf, Path partitionsPath,
184 List<ImmutableBytesWritable> startKeys) throws IOException {
185 Preconditions.checkArgument(!startKeys.isEmpty(), "No regions passed");
186
187
188
189
190
191 TreeSet<ImmutableBytesWritable> sorted =
192 new TreeSet<ImmutableBytesWritable>(startKeys);
193
194 ImmutableBytesWritable first = sorted.first();
195 Preconditions.checkArgument(
196 first.equals(HConstants.EMPTY_BYTE_ARRAY),
197 "First region of table should have empty start key. Instead has: %s",
198 Bytes.toStringBinary(first.get()));
199 sorted.remove(first);
200
201
202 FileSystem fs = partitionsPath.getFileSystem(conf);
203 SequenceFile.Writer writer = SequenceFile.createWriter(fs,
204 conf, partitionsPath, ImmutableBytesWritable.class, NullWritable.class);
205
206 try {
207 for (ImmutableBytesWritable startKey : sorted) {
208 writer.append(startKey, NullWritable.get());
209 }
210 } finally {
211 writer.close();
212 }
213 }
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229 public static void configureIncrementalLoad(Job job, HTable table) throws IOException {
230 Configuration conf = job.getConfiguration();
231 job.setPartitionerClass(TotalOrderPartitioner.class);
232 job.setOutputKeyClass(ImmutableBytesWritable.class);
233 job.setOutputValueClass(KeyValue.class);
234 job.setOutputFormatClass(HFileOutputFormat.class);
235
236
237
238
239 if (KeyValue.class.equals(job.getMapOutputValueClass())) {
240 job.setReducerClass(KeyValueSortReducer.class);
241 } else if (Put.class.equals(job.getMapOutputValueClass())) {
242 job.setReducerClass(PutSortReducer.class);
243 } else {
244 LOG.warn("Unknown map output value type:" + job.getMapOutputValueClass());
245 }
246
247 LOG.info("Looking up current regions for table " + table);
248 List<ImmutableBytesWritable> startKeys = getRegionStartKeys(table);
249 LOG.info("Configuring " + startKeys.size() + " reduce partitions " +
250 "to match current region count");
251 job.setNumReduceTasks(startKeys.size());
252
253 Path partitionsPath = new Path(job.getWorkingDirectory(),
254 "partitions_" + System.currentTimeMillis());
255 LOG.info("Writing partition information to " + partitionsPath);
256
257 FileSystem fs = partitionsPath.getFileSystem(conf);
258 writePartitions(conf, partitionsPath, startKeys);
259 partitionsPath.makeQualified(fs);
260 URI cacheUri;
261 try {
262 cacheUri = new URI(partitionsPath.toString() + "#" +
263 TotalOrderPartitioner.DEFAULT_PATH);
264 } catch (URISyntaxException e) {
265 throw new IOException(e);
266 }
267 DistributedCache.addCacheFile(cacheUri, conf);
268 DistributedCache.createSymlink(conf);
269
270 LOG.info("Incremental table output configured.");
271 }
272
273 }