View Javadoc

1   /**
2    * Copyright 2009 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.mapreduce;
21  
22  import java.io.IOException;
23  import java.io.UnsupportedEncodingException;
24  import java.net.URI;
25  import java.net.URISyntaxException;
26  import java.net.URLDecoder;
27  import java.net.URLEncoder;
28  import java.util.ArrayList;
29  import java.util.Collection;
30  import java.util.List;
31  import java.util.Map;
32  import java.util.TreeMap;
33  import java.util.TreeSet;
34  
35  import org.apache.commons.logging.Log;
36  import org.apache.commons.logging.LogFactory;
37  import org.apache.hadoop.conf.Configuration;
38  import org.apache.hadoop.filecache.DistributedCache;
39  import org.apache.hadoop.fs.FileSystem;
40  import org.apache.hadoop.fs.Path;
41  import org.apache.hadoop.hbase.HColumnDescriptor;
42  import org.apache.hadoop.hbase.HConstants;
43  import org.apache.hadoop.hbase.HTableDescriptor;
44  import org.apache.hadoop.hbase.KeyValue;
45  import org.apache.hadoop.hbase.client.HTable;
46  import org.apache.hadoop.hbase.client.Put;
47  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
48  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
49  import org.apache.hadoop.hbase.io.hfile.Compression;
50  import org.apache.hadoop.hbase.io.hfile.HFile;
51  import org.apache.hadoop.hbase.regionserver.StoreFile;
52  import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
53  import org.apache.hadoop.hbase.util.Bytes;
54  import org.apache.hadoop.io.NullWritable;
55  import org.apache.hadoop.io.SequenceFile;
56  import org.apache.hadoop.io.WritableUtils;
57  import org.apache.hadoop.mapreduce.Job;
58  import org.apache.hadoop.mapreduce.Partitioner;
59  import org.apache.hadoop.mapreduce.RecordWriter;
60  import org.apache.hadoop.mapreduce.TaskAttemptContext;
61  import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
62  import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
63  
64  /**
65   * Writes HFiles. Passed KeyValues must arrive in order.
66   * Currently, can only write files to a single column family at a
67   * time.  Multiple column families requires coordinating keys cross family.
68   * Writes current time as the sequence id for the file. Sets the major compacted
69   * attribute on created hfiles. Calling write(null,null) will forceably roll
70   * all HFiles being written.
71   * @see KeyValueSortReducer
72   */
73  public class HFileOutputFormat extends FileOutputFormat<ImmutableBytesWritable, KeyValue> {
74    static Log LOG = LogFactory.getLog(HFileOutputFormat.class);
75    static final String COMPRESSION_CONF_KEY = "hbase.hfileoutputformat.families.compression";
76    TimeRangeTracker trt = new TimeRangeTracker();
77  
78    public RecordWriter<ImmutableBytesWritable, KeyValue> getRecordWriter(final TaskAttemptContext context)
79    throws IOException, InterruptedException {
80      // Get the path of the temporary output file
81      final Path outputPath = FileOutputFormat.getOutputPath(context);
82      final Path outputdir = new FileOutputCommitter(outputPath, context).getWorkPath();
83      final Configuration conf = context.getConfiguration();
84      final FileSystem fs = outputdir.getFileSystem(conf);
85      // These configs. are from hbase-*.xml
86      final long maxsize = conf.getLong("hbase.hregion.max.filesize",
87          HConstants.DEFAULT_MAX_FILE_SIZE);
88      final int blocksize = conf.getInt("hbase.mapreduce.hfileoutputformat.blocksize",
89          HFile.DEFAULT_BLOCKSIZE);
90      // Invented config.  Add to hbase-*.xml if other than default compression.
91      final String defaultCompression = conf.get("hfile.compression",
92          Compression.Algorithm.NONE.getName());
93  
94      // create a map from column family to the compression algorithm
95      final Map<byte[], String> compressionMap = createFamilyCompressionMap(conf);
96  
97      return new RecordWriter<ImmutableBytesWritable, KeyValue>() {
98        // Map of families to writers and how much has been output on the writer.
99        private final Map<byte [], WriterLength> writers =
100         new TreeMap<byte [], WriterLength>(Bytes.BYTES_COMPARATOR);
101       private byte [] previousRow = HConstants.EMPTY_BYTE_ARRAY;
102       private final byte [] now = Bytes.toBytes(System.currentTimeMillis());
103       private boolean rollRequested = false;
104 
105       public void write(ImmutableBytesWritable row, KeyValue kv)
106       throws IOException {
107         // null input == user explicitly wants to flush
108         if (row == null && kv == null) {
109           rollWriters();
110           return;
111         }
112 
113         byte [] rowKey = kv.getRow();
114         long length = kv.getLength();
115         byte [] family = kv.getFamily();
116         WriterLength wl = this.writers.get(family);
117 
118         // If this is a new column family, verify that the directory exists
119         if (wl == null) {
120           fs.mkdirs(new Path(outputdir, Bytes.toString(family)));
121         }
122 
123         // If any of the HFiles for the column families has reached
124         // maxsize, we need to roll all the writers
125         if (wl != null && wl.written + length >= maxsize) {
126           this.rollRequested = true;
127         }
128 
129         // This can only happen once a row is finished though
130         if (rollRequested && Bytes.compareTo(this.previousRow, rowKey) != 0) {
131           rollWriters();
132         }
133 
134         // create a new HLog writer, if necessary
135         if (wl == null || wl.writer == null) {
136           wl = getNewWriter(family, conf);
137         }
138 
139         // we now have the proper HLog writer. full steam ahead
140         kv.updateLatestStamp(this.now);
141         trt.includeTimestamp(kv);
142         wl.writer.append(kv);
143         wl.written += length;
144 
145         // Copy the row so we know when a row transition.
146         this.previousRow = rowKey;
147       }
148 
149       private void rollWriters() throws IOException {
150         for (WriterLength wl : this.writers.values()) {
151           if (wl.writer != null) {
152             LOG.info("Writer=" + wl.writer.getPath() +
153                 ((wl.written == 0)? "": ", wrote=" + wl.written));
154             close(wl.writer);
155           }
156           wl.writer = null;
157           wl.written = 0;
158         }
159         this.rollRequested = false;
160       }
161 
162       /* Create a new HFile.Writer.
163        * @param family
164        * @return A WriterLength, containing a new HFile.Writer.
165        * @throws IOException
166        */
167       private WriterLength getNewWriter(byte[] family, Configuration conf)
168           throws IOException {
169         WriterLength wl = new WriterLength();
170         Path familydir = new Path(outputdir, Bytes.toString(family));
171         String compression = compressionMap.get(family);
172         compression = compression == null ? defaultCompression : compression;
173         wl.writer =
174           HFile.getWriterFactory(conf).createWriter(fs,
175           StoreFile.getUniqueFile(fs, familydir), blocksize,
176           compression, KeyValue.KEY_COMPARATOR);
177         this.writers.put(family, wl);
178         return wl;
179       }
180 
181       private void close(final HFile.Writer w) throws IOException {
182         if (w != null) {
183           w.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY,
184               Bytes.toBytes(System.currentTimeMillis()));
185           w.appendFileInfo(StoreFile.BULKLOAD_TASK_KEY,
186               Bytes.toBytes(context.getTaskAttemptID().toString()));
187           w.appendFileInfo(StoreFile.MAJOR_COMPACTION_KEY,
188               Bytes.toBytes(true));
189           w.appendFileInfo(StoreFile.TIMERANGE_KEY,
190               WritableUtils.toByteArray(trt));
191           w.close();
192         }
193       }
194 
195       public void close(TaskAttemptContext c)
196       throws IOException, InterruptedException {
197         for (WriterLength wl: this.writers.values()) {
198           close(wl.writer);
199         }
200       }
201     };
202   }
203 
204   /*
205    * Data structure to hold a Writer and amount of data written on it.
206    */
207   static class WriterLength {
208     long written = 0;
209     HFile.Writer writer = null;
210   }
211 
212   /**
213    * Return the start keys of all of the regions in this table,
214    * as a list of ImmutableBytesWritable.
215    */
216   private static List<ImmutableBytesWritable> getRegionStartKeys(HTable table)
217   throws IOException {
218     byte[][] byteKeys = table.getStartKeys();
219     ArrayList<ImmutableBytesWritable> ret =
220       new ArrayList<ImmutableBytesWritable>(byteKeys.length);
221     for (byte[] byteKey : byteKeys) {
222       ret.add(new ImmutableBytesWritable(byteKey));
223     }
224     return ret;
225   }
226 
227   /**
228    * Write out a SequenceFile that can be read by TotalOrderPartitioner
229    * that contains the split points in startKeys.
230    * @param partitionsPath output path for SequenceFile
231    * @param startKeys the region start keys
232    */
233   private static void writePartitions(Configuration conf, Path partitionsPath,
234       List<ImmutableBytesWritable> startKeys) throws IOException {
235     if (startKeys.isEmpty()) {
236       throw new IllegalArgumentException("No regions passed");
237     }
238 
239     // We're generating a list of split points, and we don't ever
240     // have keys < the first region (which has an empty start key)
241     // so we need to remove it. Otherwise we would end up with an
242     // empty reducer with index 0
243     TreeSet<ImmutableBytesWritable> sorted =
244       new TreeSet<ImmutableBytesWritable>(startKeys);
245 
246     ImmutableBytesWritable first = sorted.first();
247     if (!first.equals(HConstants.EMPTY_BYTE_ARRAY)) {
248       throw new IllegalArgumentException(
249           "First region of table should have empty start key. Instead has: "
250           + Bytes.toStringBinary(first.get()));
251     }
252     sorted.remove(first);
253     
254     // Write the actual file
255     FileSystem fs = partitionsPath.getFileSystem(conf);
256     SequenceFile.Writer writer = SequenceFile.createWriter(fs, 
257         conf, partitionsPath, ImmutableBytesWritable.class, NullWritable.class);
258     
259     try {
260       for (ImmutableBytesWritable startKey : sorted) {
261         writer.append(startKey, NullWritable.get());
262       }
263     } finally {
264       writer.close();
265     }
266   }
267   
268   /**
269    * Configure a MapReduce Job to perform an incremental load into the given
270    * table. This
271    * <ul>
272    *   <li>Inspects the table to configure a total order partitioner</li>
273    *   <li>Uploads the partitions file to the cluster and adds it to the DistributedCache</li>
274    *   <li>Sets the number of reduce tasks to match the current number of regions</li>
275    *   <li>Sets the output key/value class to match HFileOutputFormat's requirements</li>
276    *   <li>Sets the reducer up to perform the appropriate sorting (either KeyValueSortReducer or
277    *     PutSortReducer)</li>
278    * </ul> 
279    * The user should be sure to set the map output value class to either KeyValue or Put before
280    * running this function.
281    */
282   public static void configureIncrementalLoad(Job job, HTable table)
283   throws IOException {
284     Configuration conf = job.getConfiguration();
285     Class<? extends Partitioner> topClass;
286     try {
287       topClass = getTotalOrderPartitionerClass();
288     } catch (ClassNotFoundException e) {
289       throw new IOException("Failed getting TotalOrderPartitioner", e);
290     }
291     job.setPartitionerClass(topClass);
292     job.setOutputKeyClass(ImmutableBytesWritable.class);
293     job.setOutputValueClass(KeyValue.class);
294     job.setOutputFormatClass(HFileOutputFormat.class);
295 
296     // Based on the configured map output class, set the correct reducer to properly
297     // sort the incoming values.
298     // TODO it would be nice to pick one or the other of these formats.
299     if (KeyValue.class.equals(job.getMapOutputValueClass())) {
300       job.setReducerClass(KeyValueSortReducer.class);
301     } else if (Put.class.equals(job.getMapOutputValueClass())) {
302       job.setReducerClass(PutSortReducer.class);
303     } else {
304       LOG.warn("Unknown map output value type:" + job.getMapOutputValueClass());
305     }
306 
307     LOG.info("Looking up current regions for table " + table);
308     List<ImmutableBytesWritable> startKeys = getRegionStartKeys(table);
309     LOG.info("Configuring " + startKeys.size() + " reduce partitions " +
310         "to match current region count");
311     job.setNumReduceTasks(startKeys.size());
312     
313     Path partitionsPath = new Path(job.getWorkingDirectory(),
314         "partitions_" + System.currentTimeMillis());
315     LOG.info("Writing partition information to " + partitionsPath);
316 
317     FileSystem fs = partitionsPath.getFileSystem(conf);
318     writePartitions(conf, partitionsPath, startKeys);
319     partitionsPath.makeQualified(fs);
320     
321     URI cacheUri;
322     try {
323       // Below we make explicit reference to the bundled TOP.  Its cheating.
324       // We are assume the define in the hbase bundled TOP is as it is in
325       // hadoop (whether 0.20 or 0.22, etc.)
326       cacheUri = new URI(partitionsPath.toString() + "#" +
327         org.apache.hadoop.hbase.mapreduce.hadoopbackport.TotalOrderPartitioner.DEFAULT_PATH);
328     } catch (URISyntaxException e) {
329       throw new IOException(e);
330     }
331     DistributedCache.addCacheFile(cacheUri, conf);
332     DistributedCache.createSymlink(conf);
333     
334     // Set compression algorithms based on column families
335     configureCompression(table, conf);
336     
337     LOG.info("Incremental table output configured.");
338   }
339 
340   /**
341    * If > hadoop 0.20, then we want to use the hadoop TotalOrderPartitioner.
342    * If 0.20, then we want to use the TOP that we have under hadoopbackport.
343    * This method is about hbase being able to run on different versions of
344    * hadoop.  In 0.20.x hadoops, we have to use the TOP that is bundled with
345    * hbase.  Otherwise, we use the one in Hadoop.
346    * @return Instance of the TotalOrderPartitioner class
347    * @throws ClassNotFoundException If can't find a TotalOrderPartitioner.
348    */
349   private static Class<? extends Partitioner> getTotalOrderPartitionerClass()
350   throws ClassNotFoundException {
351     Class<? extends Partitioner> clazz = null;
352     try {
353       clazz = (Class<? extends Partitioner>) Class.forName("org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner");
354     } catch (ClassNotFoundException e) {
355       clazz =
356         (Class<? extends Partitioner>) Class.forName("org.apache.hadoop.hbase.mapreduce.hadoopbackport.TotalOrderPartitioner");
357     }
358     return clazz;
359   }
360 
361   /**
362    * Run inside the task to deserialize column family to compression algorithm
363    * map from the
364    * configuration.
365    * 
366    * Package-private for unit tests only.
367    * 
368    * @return a map from column family to the name of the configured compression
369    *         algorithm
370    */
371   static Map<byte[], String> createFamilyCompressionMap(Configuration conf) {
372     Map<byte[], String> compressionMap = new TreeMap<byte[], String>(Bytes.BYTES_COMPARATOR);
373     String compressionConf = conf.get(COMPRESSION_CONF_KEY, "");
374     for (String familyConf : compressionConf.split("&")) {
375       String[] familySplit = familyConf.split("=");
376       if (familySplit.length != 2) {
377         continue;
378       }
379       
380       try {
381         compressionMap.put(URLDecoder.decode(familySplit[0], "UTF-8").getBytes(),
382             URLDecoder.decode(familySplit[1], "UTF-8"));
383       } catch (UnsupportedEncodingException e) {
384         // will not happen with UTF-8 encoding
385         throw new AssertionError(e);
386       }
387     }
388     return compressionMap;
389   }
390 
391   /**
392    * Serialize column family to compression algorithm map to configuration.
393    * Invoked while configuring the MR job for incremental load.
394    * 
395    * Package-private for unit tests only.
396    * 
397    * @throws IOException
398    *           on failure to read column family descriptors
399    */
400   static void configureCompression(HTable table, Configuration conf) throws IOException {
401     StringBuilder compressionConfigValue = new StringBuilder();
402     HTableDescriptor tableDescriptor = table.getTableDescriptor();
403     if(tableDescriptor == null){
404       // could happen with mock table instance
405       return;
406     }
407     Collection<HColumnDescriptor> families = tableDescriptor.getFamilies();
408     int i = 0;
409     for (HColumnDescriptor familyDescriptor : families) {
410       if (i++ > 0) {
411         compressionConfigValue.append('&');
412       }
413       compressionConfigValue.append(URLEncoder.encode(familyDescriptor.getNameAsString(), "UTF-8"));
414       compressionConfigValue.append('=');
415       compressionConfigValue.append(URLEncoder.encode(familyDescriptor.getCompression().getName(), "UTF-8"));
416     }
417     // Get rid of the last ampersand
418     conf.set(COMPRESSION_CONF_KEY, compressionConfigValue.toString());
419   }
420 }