1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.mapreduce;
20
21 import java.io.IOException;
22 import java.io.UnsupportedEncodingException;
23 import java.net.URLDecoder;
24 import java.net.URLEncoder;
25 import java.util.ArrayList;
26 import java.util.Collection;
27 import java.util.List;
28 import java.util.Map;
29 import java.util.TreeMap;
30 import java.util.TreeSet;
31 import java.util.UUID;
32
33 import org.apache.commons.logging.Log;
34 import org.apache.commons.logging.LogFactory;
35 import org.apache.hadoop.classification.InterfaceAudience;
36 import org.apache.hadoop.classification.InterfaceStability;
37 import org.apache.hadoop.conf.Configuration;
38 import org.apache.hadoop.fs.FileSystem;
39 import org.apache.hadoop.fs.Path;
40 import org.apache.hadoop.hbase.HColumnDescriptor;
41 import org.apache.hadoop.hbase.HConstants;
42 import org.apache.hadoop.hbase.HTableDescriptor;
43 import org.apache.hadoop.hbase.KeyValue;
44 import org.apache.hadoop.hbase.client.HTable;
45 import org.apache.hadoop.hbase.client.Put;
46 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
47 import org.apache.hadoop.hbase.io.compress.Compression;
48 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
49 import org.apache.hadoop.hbase.io.hfile.AbstractHFileWriter;
50 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
51 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
52 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl;
53 import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder;
54 import org.apache.hadoop.hbase.regionserver.BloomType;
55 import org.apache.hadoop.hbase.regionserver.HStore;
56 import org.apache.hadoop.hbase.regionserver.StoreFile;
57 import org.apache.hadoop.hbase.util.Bytes;
58 import org.apache.hadoop.io.NullWritable;
59 import org.apache.hadoop.io.SequenceFile;
60 import org.apache.hadoop.mapreduce.Job;
61 import org.apache.hadoop.mapreduce.RecordWriter;
62 import org.apache.hadoop.mapreduce.TaskAttemptContext;
63 import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
64 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
65 import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
66
67
68
69
70
71
72
73
74
75
76
77 @InterfaceAudience.Public
78 @InterfaceStability.Stable
79 public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable, KeyValue> {
80 static Log LOG = LogFactory.getLog(HFileOutputFormat.class);
81 static final String COMPRESSION_CONF_KEY = "hbase.hfileoutputformat.families.compression";
82 private static final String BLOOM_TYPE_CONF_KEY = "hbase.hfileoutputformat.families.bloomtype";
83 private static final String DATABLOCK_ENCODING_CONF_KEY =
84 "hbase.mapreduce.hfileoutputformat.datablock.encoding";
85
86 public RecordWriter<ImmutableBytesWritable, KeyValue> getRecordWriter(final TaskAttemptContext context)
87 throws IOException, InterruptedException {
88
89 final Path outputPath = FileOutputFormat.getOutputPath(context);
90 final Path outputdir = new FileOutputCommitter(outputPath, context).getWorkPath();
91 final Configuration conf = context.getConfiguration();
92 final FileSystem fs = outputdir.getFileSystem(conf);
93
94 final long maxsize = conf.getLong(HConstants.HREGION_MAX_FILESIZE,
95 HConstants.DEFAULT_MAX_FILE_SIZE);
96 final int blocksize = conf.getInt("hbase.mapreduce.hfileoutputformat.blocksize",
97 HConstants.DEFAULT_BLOCKSIZE);
98
99 final String defaultCompression = conf.get("hfile.compression",
100 Compression.Algorithm.NONE.getName());
101 final boolean compactionExclude = conf.getBoolean(
102 "hbase.mapreduce.hfileoutputformat.compaction.exclude", false);
103
104
105 final Map<byte[], String> compressionMap = createFamilyCompressionMap(conf);
106 final Map<byte[], String> bloomTypeMap = createFamilyBloomMap(conf);
107
108 String dataBlockEncodingStr = conf.get(DATABLOCK_ENCODING_CONF_KEY);
109 final HFileDataBlockEncoder encoder;
110 if (dataBlockEncodingStr == null) {
111 encoder = NoOpDataBlockEncoder.INSTANCE;
112 } else {
113 try {
114 encoder = new HFileDataBlockEncoderImpl(DataBlockEncoding
115 .valueOf(dataBlockEncodingStr));
116 } catch (IllegalArgumentException ex) {
117 throw new RuntimeException(
118 "Invalid data block encoding type configured for the param "
119 + DATABLOCK_ENCODING_CONF_KEY + " : " + dataBlockEncodingStr);
120 }
121 }
122
123 return new RecordWriter<ImmutableBytesWritable, KeyValue>() {
124
125 private final Map<byte [], WriterLength> writers =
126 new TreeMap<byte [], WriterLength>(Bytes.BYTES_COMPARATOR);
127 private byte [] previousRow = HConstants.EMPTY_BYTE_ARRAY;
128 private final byte [] now = Bytes.toBytes(System.currentTimeMillis());
129 private boolean rollRequested = false;
130
131 public void write(ImmutableBytesWritable row, KeyValue kv)
132 throws IOException {
133
134 if (row == null && kv == null) {
135 rollWriters();
136 return;
137 }
138
139 byte [] rowKey = kv.getRow();
140 long length = kv.getLength();
141 byte [] family = kv.getFamily();
142 WriterLength wl = this.writers.get(family);
143
144
145 if (wl == null) {
146 fs.mkdirs(new Path(outputdir, Bytes.toString(family)));
147 }
148
149
150
151 if (wl != null && wl.written + length >= maxsize) {
152 this.rollRequested = true;
153 }
154
155
156 if (rollRequested && Bytes.compareTo(this.previousRow, rowKey) != 0) {
157 rollWriters();
158 }
159
160
161 if (wl == null || wl.writer == null) {
162 wl = getNewWriter(family, conf);
163 }
164
165
166 kv.updateLatestStamp(this.now);
167 wl.writer.append(kv);
168 wl.written += length;
169
170
171 this.previousRow = rowKey;
172 }
173
174 private void rollWriters() throws IOException {
175 for (WriterLength wl : this.writers.values()) {
176 if (wl.writer != null) {
177 LOG.info("Writer=" + wl.writer.getPath() +
178 ((wl.written == 0)? "": ", wrote=" + wl.written));
179 close(wl.writer);
180 }
181 wl.writer = null;
182 wl.written = 0;
183 }
184 this.rollRequested = false;
185 }
186
187
188
189
190
191
192 private WriterLength getNewWriter(byte[] family, Configuration conf)
193 throws IOException {
194 WriterLength wl = new WriterLength();
195 Path familydir = new Path(outputdir, Bytes.toString(family));
196 String compression = compressionMap.get(family);
197 compression = compression == null ? defaultCompression : compression;
198 String bloomTypeStr = bloomTypeMap.get(family);
199 BloomType bloomType = BloomType.NONE;
200 if (bloomTypeStr != null) {
201 bloomType = BloomType.valueOf(bloomTypeStr);
202 }
203 Configuration tempConf = new Configuration(conf);
204 tempConf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
205 wl.writer = new StoreFile.WriterBuilder(conf, new CacheConfig(tempConf), fs, blocksize)
206 .withOutputDir(familydir)
207 .withCompression(AbstractHFileWriter.compressionByName(compression))
208 .withBloomType(bloomType)
209 .withComparator(KeyValue.COMPARATOR)
210 .withDataBlockEncoder(encoder)
211 .withChecksumType(HStore.getChecksumType(conf))
212 .withBytesPerChecksum(HStore.getBytesPerChecksum(conf))
213 .build();
214
215 this.writers.put(family, wl);
216 return wl;
217 }
218
219 private void close(final StoreFile.Writer w) throws IOException {
220 if (w != null) {
221 w.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY,
222 Bytes.toBytes(System.currentTimeMillis()));
223 w.appendFileInfo(StoreFile.BULKLOAD_TASK_KEY,
224 Bytes.toBytes(context.getTaskAttemptID().toString()));
225 w.appendFileInfo(StoreFile.MAJOR_COMPACTION_KEY,
226 Bytes.toBytes(true));
227 w.appendFileInfo(StoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY,
228 Bytes.toBytes(compactionExclude));
229 w.appendTrackedTimestampsToMetadata();
230 w.close();
231 }
232 }
233
234 public void close(TaskAttemptContext c)
235 throws IOException, InterruptedException {
236 for (WriterLength wl: this.writers.values()) {
237 close(wl.writer);
238 }
239 }
240 };
241 }
242
243
244
245
246 static class WriterLength {
247 long written = 0;
248 StoreFile.Writer writer = null;
249 }
250
251
252
253
254
255 private static List<ImmutableBytesWritable> getRegionStartKeys(HTable table)
256 throws IOException {
257 byte[][] byteKeys = table.getStartKeys();
258 ArrayList<ImmutableBytesWritable> ret =
259 new ArrayList<ImmutableBytesWritable>(byteKeys.length);
260 for (byte[] byteKey : byteKeys) {
261 ret.add(new ImmutableBytesWritable(byteKey));
262 }
263 return ret;
264 }
265
266
267
268
269
270 private static void writePartitions(Configuration conf, Path partitionsPath,
271 List<ImmutableBytesWritable> startKeys) throws IOException {
272 LOG.info("Writing partition information to " + partitionsPath);
273 if (startKeys.isEmpty()) {
274 throw new IllegalArgumentException("No regions passed");
275 }
276
277
278
279
280
281 TreeSet<ImmutableBytesWritable> sorted =
282 new TreeSet<ImmutableBytesWritable>(startKeys);
283
284 ImmutableBytesWritable first = sorted.first();
285 if (!first.equals(HConstants.EMPTY_BYTE_ARRAY)) {
286 throw new IllegalArgumentException(
287 "First region of table should have empty start key. Instead has: "
288 + Bytes.toStringBinary(first.get()));
289 }
290 sorted.remove(first);
291
292
293 FileSystem fs = partitionsPath.getFileSystem(conf);
294 SequenceFile.Writer writer = SequenceFile.createWriter(fs,
295 conf, partitionsPath, ImmutableBytesWritable.class, NullWritable.class);
296
297 try {
298 for (ImmutableBytesWritable startKey : sorted) {
299 writer.append(startKey, NullWritable.get());
300 }
301 } finally {
302 writer.close();
303 }
304 }
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320 public static void configureIncrementalLoad(Job job, HTable table)
321 throws IOException {
322 Configuration conf = job.getConfiguration();
323
324 job.setOutputKeyClass(ImmutableBytesWritable.class);
325 job.setOutputValueClass(KeyValue.class);
326 job.setOutputFormatClass(HFileOutputFormat.class);
327
328
329
330
331 if (KeyValue.class.equals(job.getMapOutputValueClass())) {
332 job.setReducerClass(KeyValueSortReducer.class);
333 } else if (Put.class.equals(job.getMapOutputValueClass())) {
334 job.setReducerClass(PutSortReducer.class);
335 } else {
336 LOG.warn("Unknown map output value type:" + job.getMapOutputValueClass());
337 }
338
339
340 LOG.info("Looking up current regions for table " + table);
341 List<ImmutableBytesWritable> startKeys = getRegionStartKeys(table);
342 LOG.info("Configuring " + startKeys.size() + " reduce partitions " +
343 "to match current region count");
344 job.setNumReduceTasks(startKeys.size());
345
346 configurePartitioner(job, startKeys);
347
348 configureCompression(table, conf);
349 configureBloomType(table, conf);
350
351 TableMapReduceUtil.addDependencyJars(job);
352 LOG.info("Incremental table output configured.");
353 }
354
355
356
357
358
359
360
361
362
363
364
365 static Map<byte[], String> createFamilyCompressionMap(Configuration conf) {
366 return createFamilyConfValueMap(conf, COMPRESSION_CONF_KEY);
367 }
368
369 private static Map<byte[], String> createFamilyBloomMap(Configuration conf) {
370 return createFamilyConfValueMap(conf, BLOOM_TYPE_CONF_KEY);
371 }
372
373
374
375
376
377
378
379
380 private static Map<byte[], String> createFamilyConfValueMap(Configuration conf, String confName) {
381 Map<byte[], String> confValMap = new TreeMap<byte[], String>(Bytes.BYTES_COMPARATOR);
382 String confVal = conf.get(confName, "");
383 for (String familyConf : confVal.split("&")) {
384 String[] familySplit = familyConf.split("=");
385 if (familySplit.length != 2) {
386 continue;
387 }
388 try {
389 confValMap.put(URLDecoder.decode(familySplit[0], "UTF-8").getBytes(),
390 URLDecoder.decode(familySplit[1], "UTF-8"));
391 } catch (UnsupportedEncodingException e) {
392
393 throw new AssertionError(e);
394 }
395 }
396 return confValMap;
397 }
398
399
400
401
402
403 static void configurePartitioner(Job job, List<ImmutableBytesWritable> splitPoints)
404 throws IOException {
405
406
407 FileSystem fs = FileSystem.get(job.getConfiguration());
408 Path partitionsPath = new Path("/tmp", "partitions_" + UUID.randomUUID());
409 fs.makeQualified(partitionsPath);
410 fs.deleteOnExit(partitionsPath);
411 writePartitions(job.getConfiguration(), partitionsPath, splitPoints);
412
413
414 job.setPartitionerClass(TotalOrderPartitioner.class);
415 TotalOrderPartitioner.setPartitionFile(job.getConfiguration(), partitionsPath);
416 }
417
418
419
420
421
422
423
424
425
426
427 @edu.umd.cs.findbugs.annotations.SuppressWarnings(
428 value="RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE")
429 static void configureCompression(HTable table, Configuration conf) throws IOException {
430 StringBuilder compressionConfigValue = new StringBuilder();
431 HTableDescriptor tableDescriptor = table.getTableDescriptor();
432 if(tableDescriptor == null){
433
434 return;
435 }
436 Collection<HColumnDescriptor> families = tableDescriptor.getFamilies();
437 int i = 0;
438 for (HColumnDescriptor familyDescriptor : families) {
439 if (i++ > 0) {
440 compressionConfigValue.append('&');
441 }
442 compressionConfigValue.append(URLEncoder.encode(familyDescriptor.getNameAsString(), "UTF-8"));
443 compressionConfigValue.append('=');
444 compressionConfigValue.append(URLEncoder.encode(familyDescriptor.getCompression().getName(), "UTF-8"));
445 }
446
447 conf.set(COMPRESSION_CONF_KEY, compressionConfigValue.toString());
448 }
449
450
451
452
453
454
455
456
457 static void configureBloomType(HTable table, Configuration conf) throws IOException {
458 HTableDescriptor tableDescriptor = table.getTableDescriptor();
459 if (tableDescriptor == null) {
460
461 return;
462 }
463 StringBuilder bloomTypeConfigValue = new StringBuilder();
464 Collection<HColumnDescriptor> families = tableDescriptor.getFamilies();
465 int i = 0;
466 for (HColumnDescriptor familyDescriptor : families) {
467 if (i++ > 0) {
468 bloomTypeConfigValue.append('&');
469 }
470 bloomTypeConfigValue.append(URLEncoder.encode(familyDescriptor.getNameAsString(), "UTF-8"));
471 bloomTypeConfigValue.append('=');
472 String bloomType = familyDescriptor.getBloomFilterType().toString();
473 if (bloomType == null) {
474 bloomType = HColumnDescriptor.DEFAULT_BLOOMFILTER;
475 }
476 bloomTypeConfigValue.append(URLEncoder.encode(bloomType, "UTF-8"));
477 }
478 conf.set(BLOOM_TYPE_CONF_KEY, bloomTypeConfigValue.toString());
479 }
480 }