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 static org.junit.Assert.assertEquals;
23  import static org.junit.Assert.assertFalse;
24  import static org.junit.Assert.assertNotNull;
25  import static org.junit.Assert.assertNotSame;
26  import static org.junit.Assert.assertTrue;
27  import static org.junit.Assert.fail;
28  
29  import java.io.IOException;
30  import java.lang.reflect.Constructor;
31  import java.util.Arrays;
32  import java.util.HashMap;
33  import java.util.List;
34  import java.util.Map;
35  import java.util.Map.Entry;
36  import java.util.Random;
37  
38  import junit.framework.Assert;
39  import org.apache.commons.logging.Log;
40  import org.apache.commons.logging.LogFactory;
41  import org.apache.hadoop.conf.Configuration;
42  import org.apache.hadoop.fs.FileStatus;
43  import org.apache.hadoop.fs.FileSystem;
44  import org.apache.hadoop.fs.Path;
45  import org.apache.hadoop.hbase.HBaseConfiguration;
46  import org.apache.hadoop.hbase.HBaseTestingUtility;
47  import org.apache.hadoop.hbase.HColumnDescriptor;
48  import org.apache.hadoop.hbase.HConstants;
49  import org.apache.hadoop.hbase.HTableDescriptor;
50  import org.apache.hadoop.hbase.KeyValue;
51  import org.apache.hadoop.hbase.PerformanceEvaluation;
52  import org.apache.hadoop.hbase.client.HBaseAdmin;
53  import org.apache.hadoop.hbase.client.HTable;
54  import org.apache.hadoop.hbase.client.Result;
55  import org.apache.hadoop.hbase.client.ResultScanner;
56  import org.apache.hadoop.hbase.client.Scan;
57  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
58  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
59  import org.apache.hadoop.hbase.io.hfile.Compression;
60  import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
61  import org.apache.hadoop.hbase.io.hfile.HFile;
62  import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
63  import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
64  import org.apache.hadoop.hbase.util.Bytes;
65  import org.apache.hadoop.hbase.util.Threads;
66  import org.apache.hadoop.hbase.util.Writables;
67  import org.apache.hadoop.io.NullWritable;
68  import org.apache.hadoop.mapreduce.Job;
69  import org.apache.hadoop.mapreduce.Mapper;
70  import org.apache.hadoop.mapreduce.RecordWriter;
71  import org.apache.hadoop.mapreduce.TaskAttemptContext;
72  import org.apache.hadoop.mapreduce.TaskAttemptID;
73  import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
74  import org.junit.Before;
75  import org.junit.Test;
76  import org.mockito.Mockito;
77  
78  import com.google.common.collect.Lists;
79  
80  /**
81   * Simple test for {@link KeyValueSortReducer} and {@link HFileOutputFormat}.
82   * Sets up and runs a mapreduce job that writes hfile output.
83   * Creates a few inner classes to implement splits and an inputformat that
84   * emits keys and values like those of {@link PerformanceEvaluation}.
85   */
86  public class TestHFileOutputFormat  {
87    private final static int ROWSPERSPLIT = 1024;
88  
89    private static final byte[][] FAMILIES
90      = { Bytes.add(PerformanceEvaluation.FAMILY_NAME, Bytes.toBytes("-A"))
91        , Bytes.add(PerformanceEvaluation.FAMILY_NAME, Bytes.toBytes("-B"))};
92    private static final byte[] TABLE_NAME = Bytes.toBytes("TestTable");
93    
94    private HBaseTestingUtility util = new HBaseTestingUtility();
95    
96    private static Log LOG = LogFactory.getLog(TestHFileOutputFormat.class);
97    
98    /**
99     * Simple mapper that makes KeyValue output.
100    */
101   static class RandomKVGeneratingMapper
102   extends Mapper<NullWritable, NullWritable,
103                  ImmutableBytesWritable, KeyValue> {
104     
105     private int keyLength;
106     private static final int KEYLEN_DEFAULT=10;
107     private static final String KEYLEN_CONF="randomkv.key.length";
108 
109     private int valLength;
110     private static final int VALLEN_DEFAULT=10;
111     private static final String VALLEN_CONF="randomkv.val.length";
112     
113     @Override
114     protected void setup(Context context) throws IOException,
115         InterruptedException {
116       super.setup(context);
117       
118       Configuration conf = context.getConfiguration();
119       keyLength = conf.getInt(KEYLEN_CONF, KEYLEN_DEFAULT);
120       valLength = conf.getInt(VALLEN_CONF, VALLEN_DEFAULT);
121     }
122 
123     protected void map(
124         NullWritable n1, NullWritable n2,
125         Mapper<NullWritable, NullWritable,
126                ImmutableBytesWritable,KeyValue>.Context context)
127         throws java.io.IOException ,InterruptedException
128     {
129 
130       byte keyBytes[] = new byte[keyLength];
131       byte valBytes[] = new byte[valLength];
132       
133       int taskId = context.getTaskAttemptID().getTaskID().getId();
134       assert taskId < Byte.MAX_VALUE : "Unit tests dont support > 127 tasks!";
135 
136       Random random = new Random();
137       for (int i = 0; i < ROWSPERSPLIT; i++) {
138 
139         random.nextBytes(keyBytes);
140         // Ensure that unique tasks generate unique keys
141         keyBytes[keyLength - 1] = (byte)(taskId & 0xFF);
142         random.nextBytes(valBytes);
143         ImmutableBytesWritable key = new ImmutableBytesWritable(keyBytes);
144 
145         for (byte[] family : TestHFileOutputFormat.FAMILIES) {
146           KeyValue kv = new KeyValue(keyBytes, family,
147               PerformanceEvaluation.QUALIFIER_NAME, valBytes);
148           context.write(key, kv);
149         }
150       }
151     }
152   }
153 
154   @Before
155   public void cleanupDir() throws IOException {
156     util.cleanupTestDir();
157   }
158   
159   
160   private void setupRandomGeneratorMapper(Job job) {
161     job.setInputFormatClass(NMapInputFormat.class);
162     job.setMapperClass(RandomKVGeneratingMapper.class);
163     job.setMapOutputKeyClass(ImmutableBytesWritable.class);
164     job.setMapOutputValueClass(KeyValue.class);
165   }
166 
167   /**
168    * Test that {@link HFileOutputFormat} RecordWriter amends timestamps if
169    * passed a keyvalue whose timestamp is {@link HConstants#LATEST_TIMESTAMP}.
170    * @see <a href="https://issues.apache.org/jira/browse/HBASE-2615">HBASE-2615</a>
171    */
172   @Test
173   public void test_LATEST_TIMESTAMP_isReplaced()
174   throws Exception {
175     Configuration conf = new Configuration(this.util.getConfiguration());
176     RecordWriter<ImmutableBytesWritable, KeyValue> writer = null;
177     TaskAttemptContext context = null;
178     Path dir =
179       util.getDataTestDir("test_LATEST_TIMESTAMP_isReplaced");
180     try {
181       Job job = new Job(conf);
182       FileOutputFormat.setOutputPath(job, dir);
183       context = getTestTaskAttemptContext(job);
184       HFileOutputFormat hof = new HFileOutputFormat();
185       writer = hof.getRecordWriter(context);
186       final byte [] b = Bytes.toBytes("b");
187 
188       // Test 1.  Pass a KV that has a ts of LATEST_TIMESTAMP.  It should be
189       // changed by call to write.  Check all in kv is same but ts.
190       KeyValue kv = new KeyValue(b, b, b);
191       KeyValue original = kv.clone();
192       writer.write(new ImmutableBytesWritable(), kv);
193       assertFalse(original.equals(kv));
194       assertTrue(Bytes.equals(original.getRow(), kv.getRow()));
195       assertTrue(original.matchingColumn(kv.getFamily(), kv.getQualifier()));
196       assertNotSame(original.getTimestamp(), kv.getTimestamp());
197       assertNotSame(HConstants.LATEST_TIMESTAMP, kv.getTimestamp());
198 
199       // Test 2. Now test passing a kv that has explicit ts.  It should not be
200       // changed by call to record write.
201       kv = new KeyValue(b, b, b, kv.getTimestamp() - 1, b);
202       original = kv.clone();
203       writer.write(new ImmutableBytesWritable(), kv);
204       assertTrue(original.equals(kv));
205     } finally {
206       if (writer != null && context != null) writer.close(context);
207       dir.getFileSystem(conf).delete(dir, true);
208     }
209   }
210 
211   /**
212    * @return True if the available mapreduce is post-0.20.
213    */
214   private static boolean isPost020MapReduce() {
215     // Here is a coarse test for post 0.20 hadoop; TAC became an interface.
216     return TaskAttemptContext.class.isInterface();
217   }
218 
219   private TaskAttemptContext getTestTaskAttemptContext(final Job job)
220   throws IOException, Exception {
221     TaskAttemptContext context;
222     if (isPost020MapReduce()) {
223       TaskAttemptID id =
224         TaskAttemptID.forName("attempt_200707121733_0001_m_000000_0");
225       Class<?> clazz =
226         Class.forName("org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl");
227       Constructor<?> c = clazz.
228           getConstructor(Configuration.class, TaskAttemptID.class);
229       context = (TaskAttemptContext)c.newInstance(job.getConfiguration(), id);
230     } else {
231       context = org.apache.hadoop.hbase.mapreduce.hadoopbackport.InputSampler.
232         getTaskAttemptContext(job);
233     }
234     return context;
235   }
236 
237   /*
238    * Test that {@link HFileOutputFormat} creates an HFile with TIMERANGE
239    * metadata used by time-restricted scans.
240    */
241   @Test
242   public void test_TIMERANGE() throws Exception { 
243     Configuration conf = new Configuration(this.util.getConfiguration());
244     RecordWriter<ImmutableBytesWritable, KeyValue> writer = null;
245     TaskAttemptContext context = null;
246     Path dir =
247       util.getDataTestDir("test_TIMERANGE_present");
248     LOG.info("Timerange dir writing to dir: "+ dir);
249     try {
250       // build a record writer using HFileOutputFormat
251       Job job = new Job(conf);
252       FileOutputFormat.setOutputPath(job, dir);
253       context = getTestTaskAttemptContext(job);
254       HFileOutputFormat hof = new HFileOutputFormat();
255       writer = hof.getRecordWriter(context);
256 
257       // Pass two key values with explicit times stamps
258       final byte [] b = Bytes.toBytes("b");
259 
260       // value 1 with timestamp 2000
261       KeyValue kv = new KeyValue(b, b, b, 2000, b);
262       KeyValue original = kv.clone();
263       writer.write(new ImmutableBytesWritable(), kv);
264       assertEquals(original,kv);
265 
266       // value 2 with timestamp 1000
267       kv = new KeyValue(b, b, b, 1000, b);
268       original = kv.clone();
269       writer.write(new ImmutableBytesWritable(), kv);
270       assertEquals(original, kv);
271 
272       // verify that the file has the proper FileInfo.
273       writer.close(context);
274 
275       // the generated file lives 3 directories down and is the only file,
276       // so we traverse the dirs to get to the file
277       // ./_temporary/_attempt__0000_r_000000_0/b/1979617994050536795
278       FileSystem fs = FileSystem.get(conf);
279       Path path = HFileOutputFormat.getOutputPath(job);
280       FileStatus[] sub1 = fs.listStatus(path);
281       FileStatus[] sub2 = fs.listStatus(sub1[0].getPath());
282       FileStatus[] sub3 = fs.listStatus(sub2[0].getPath());
283       FileStatus[] file = fs.listStatus(sub3[0].getPath());
284 
285       // open as HFile Reader and pull out TIMERANGE FileInfo.
286       HFile.Reader rd = HFile.createReader(fs, file[0].getPath(),
287           new CacheConfig(conf));
288       Map<byte[],byte[]> finfo = rd.loadFileInfo();
289       byte[] range = finfo.get("TIMERANGE".getBytes());
290       assertNotNull(range);
291 
292       // unmarshall and check values.
293       TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
294       Writables.copyWritable(range, timeRangeTracker);
295       LOG.info(timeRangeTracker.getMinimumTimestamp() +
296           "...." + timeRangeTracker.getMaximumTimestamp());
297       assertEquals(1000, timeRangeTracker.getMinimumTimestamp());
298       assertEquals(2000, timeRangeTracker.getMaximumTimestamp());
299     } finally {
300       if (writer != null && context != null) writer.close(context);
301       dir.getFileSystem(conf).delete(dir, true);
302     }
303   }
304 
305   /**
306    * Run small MR job.
307    */
308   @Test
309   public void testWritingPEData() throws Exception {
310     Configuration conf = util.getConfiguration();
311     Path testDir = util.getDataTestDir("testWritingPEData");
312     FileSystem fs = testDir.getFileSystem(conf);
313     
314     // Set down this value or we OOME in eclipse.
315     conf.setInt("io.sort.mb", 20);
316     // Write a few files.
317     conf.setLong("hbase.hregion.max.filesize", 64 * 1024);
318     
319     Job job = new Job(conf, "testWritingPEData");
320     setupRandomGeneratorMapper(job);
321     // This partitioner doesn't work well for number keys but using it anyways
322     // just to demonstrate how to configure it.
323     byte[] startKey = new byte[RandomKVGeneratingMapper.KEYLEN_DEFAULT];
324     byte[] endKey = new byte[RandomKVGeneratingMapper.KEYLEN_DEFAULT];
325     
326     Arrays.fill(startKey, (byte)0);
327     Arrays.fill(endKey, (byte)0xff);
328     
329     job.setPartitionerClass(SimpleTotalOrderPartitioner.class);
330     // Set start and end rows for partitioner.
331     SimpleTotalOrderPartitioner.setStartKey(job.getConfiguration(), startKey);
332     SimpleTotalOrderPartitioner.setEndKey(job.getConfiguration(), endKey);
333     job.setReducerClass(KeyValueSortReducer.class);
334     job.setOutputFormatClass(HFileOutputFormat.class);
335     job.setNumReduceTasks(4);
336     
337     FileOutputFormat.setOutputPath(job, testDir);
338     assertTrue(job.waitForCompletion(false));
339     FileStatus [] files = fs.listStatus(testDir);
340     assertTrue(files.length > 0);
341   }
342   
343   @Test
344   public void testJobConfiguration() throws Exception {
345     Job job = new Job();
346     HTable table = Mockito.mock(HTable.class);
347     setupMockStartKeys(table);
348     HFileOutputFormat.configureIncrementalLoad(job, table);
349     assertEquals(job.getNumReduceTasks(), 4);
350   }
351 
352   private byte [][] generateRandomStartKeys(int numKeys) {
353     Random random = new Random();
354     byte[][] ret = new byte[numKeys][];
355     // first region start key is always empty
356     ret[0] = HConstants.EMPTY_BYTE_ARRAY;
357     for (int i = 1; i < numKeys; i++) {
358       ret[i] = PerformanceEvaluation.generateValue(random);
359     }
360     return ret;
361   }
362 
363   @Test
364   public void testMRIncrementalLoad() throws Exception {
365     doIncrementalLoadTest(false);
366   }
367 
368   @Test
369   public void testMRIncrementalLoadWithSplit() throws Exception {
370     doIncrementalLoadTest(true);
371   }
372   
373   private void doIncrementalLoadTest(
374       boolean shouldChangeRegions) throws Exception {
375     Configuration conf = util.getConfiguration();
376     Path testDir = util.getDataTestDir("testLocalMRIncrementalLoad");
377     byte[][] startKeys = generateRandomStartKeys(5);
378     
379     try {
380       util.startMiniCluster();
381       HBaseAdmin admin = new HBaseAdmin(conf);
382       HTable table = util.createTable(TABLE_NAME, FAMILIES);
383       assertEquals("Should start with empty table",
384           0, util.countRows(table));
385       int numRegions = util.createMultiRegions(
386           util.getConfiguration(), table, FAMILIES[0], startKeys);
387       assertEquals("Should make 5 regions", numRegions, 5);
388 
389       // Generate the bulk load files
390       util.startMiniMapReduceCluster();
391       runIncrementalPELoad(conf, table, testDir);
392       // This doesn't write into the table, just makes files
393       assertEquals("HFOF should not touch actual table",
394           0, util.countRows(table));
395   
396 
397       // Make sure that a directory was created for every CF
398       int dir = 0;
399       for (FileStatus f : testDir.getFileSystem(conf).listStatus(testDir)) {
400         for (byte[] family : FAMILIES) {
401           if (Bytes.toString(family).equals(f.getPath().getName())) {
402             ++dir;
403           }
404         }
405       }
406       assertEquals("Column family not found in FS.", FAMILIES.length, dir);
407 
408       // handle the split case
409       if (shouldChangeRegions) {
410         LOG.info("Changing regions in table");
411         admin.disableTable(table.getTableName());
412         while(util.getMiniHBaseCluster().getMaster().getAssignmentManager().
413             isRegionsInTransition()) {
414           Threads.sleep(200);
415           LOG.info("Waiting on table to finish disabling");
416         }
417         byte[][] newStartKeys = generateRandomStartKeys(15);
418         util.createMultiRegions(
419             util.getConfiguration(), table, FAMILIES[0], newStartKeys);
420         admin.enableTable(table.getTableName());
421         while (table.getRegionsInfo().size() != 15 ||
422             !admin.isTableAvailable(table.getTableName())) {
423           Thread.sleep(200);
424           LOG.info("Waiting for new region assignment to happen");
425         }
426       }
427       
428       // Perform the actual load
429       new LoadIncrementalHFiles(conf).doBulkLoad(testDir, table);
430       
431       // Ensure data shows up
432       int expectedRows = NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;
433       assertEquals("LoadIncrementalHFiles should put expected data in table",
434           expectedRows, util.countRows(table));
435       Scan scan = new Scan();
436       ResultScanner results = table.getScanner(scan);
437       int count = 0;
438       for (Result res : results) {
439         count++;
440         assertEquals(FAMILIES.length, res.raw().length);
441         KeyValue first = res.raw()[0];
442         for (KeyValue kv : res.raw()) {
443           assertTrue(KeyValue.COMPARATOR.matchingRows(first, kv));
444           assertTrue(Bytes.equals(first.getValue(), kv.getValue()));
445         }
446       }
447       results.close();
448       String tableDigestBefore = util.checksumRows(table);
449             
450       // Cause regions to reopen
451       admin.disableTable(TABLE_NAME);
452       while (!admin.isTableDisabled(TABLE_NAME)) {
453         Thread.sleep(200);
454         LOG.info("Waiting for table to disable"); 
455       }
456       admin.enableTable(TABLE_NAME);
457       util.waitTableAvailable(TABLE_NAME, 30000);
458       assertEquals("Data should remain after reopening of regions",
459           tableDigestBefore, util.checksumRows(table));
460     } finally {
461       util.shutdownMiniMapReduceCluster();
462       util.shutdownMiniCluster();
463     }
464   }
465 
466   private void runIncrementalPELoad(
467       Configuration conf, HTable table, Path outDir)
468   throws Exception {
469     Job job = new Job(conf, "testLocalMRIncrementalLoad");
470     setupRandomGeneratorMapper(job);
471     HFileOutputFormat.configureIncrementalLoad(job, table);
472     FileOutputFormat.setOutputPath(job, outDir);
473 
474     Assert.assertFalse( util.getTestFileSystem().exists(outDir)) ;
475 
476     assertEquals(table.getRegionsInfo().size(),
477       job.getNumReduceTasks());
478     
479     assertTrue(job.waitForCompletion(true));
480   }
481   
482   /**
483    * Test for
484    * {@link HFileOutputFormat#createFamilyCompressionMap(Configuration)}. Tests
485    * that the compression map is correctly deserialized from configuration
486    * 
487    * @throws IOException
488    */
489   @Test
490   public void testCreateFamilyCompressionMap() throws IOException {
491     for (int numCfs = 0; numCfs <= 3; numCfs++) {
492       Configuration conf = new Configuration(this.util.getConfiguration());
493       Map<String, Compression.Algorithm> familyToCompression = getMockColumnFamilies(numCfs);
494       HTable table = Mockito.mock(HTable.class);
495       setupMockColumnFamilies(table, familyToCompression);
496       HFileOutputFormat.configureCompression(table, conf);
497 
498       // read back family specific compression setting from the configuration
499       Map<byte[], String> retrievedFamilyToCompressionMap = HFileOutputFormat.createFamilyCompressionMap(conf);
500 
501       // test that we have a value for all column families that matches with the
502       // used mock values
503       for (Entry<String, Algorithm> entry : familyToCompression.entrySet()) {
504         assertEquals("Compression configuration incorrect for column family:" + entry.getKey(), entry.getValue()
505                      .getName(), retrievedFamilyToCompressionMap.get(entry.getKey().getBytes()));
506       }
507     }
508   }
509 
510   private void setupMockColumnFamilies(HTable table,
511     Map<String, Compression.Algorithm> familyToCompression) throws IOException
512   {
513     HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME);
514     for (Entry<String, Compression.Algorithm> entry : familyToCompression.entrySet()) {
515       mockTableDescriptor.addFamily(new HColumnDescriptor(entry.getKey().getBytes(), 1, entry.getValue().getName(),
516                                                           false, false, 0, "none"));
517     }
518     Mockito.doReturn(mockTableDescriptor).when(table).getTableDescriptor();
519   }
520 
521   private void setupMockStartKeys(HTable table) throws IOException {
522     byte[][] mockKeys = new byte[][] {
523         HConstants.EMPTY_BYTE_ARRAY,
524         Bytes.toBytes("aaa"),
525         Bytes.toBytes("ggg"),
526         Bytes.toBytes("zzz")
527     };
528     Mockito.doReturn(mockKeys).when(table).getStartKeys();
529   }
530 
531   /**
532    * @return a map from column family names to compression algorithms for
533    *         testing column family compression. Column family names have special characters
534    */
535   private Map<String, Compression.Algorithm> getMockColumnFamilies(int numCfs) {
536     Map<String, Compression.Algorithm> familyToCompression = new HashMap<String, Compression.Algorithm>();
537     // use column family names having special characters
538     if (numCfs-- > 0) {
539       familyToCompression.put("Family1!@#!@#&", Compression.Algorithm.LZO);
540     }
541     if (numCfs-- > 0) {
542       familyToCompression.put("Family2=asdads&!AASD", Compression.Algorithm.SNAPPY);
543     }
544     if (numCfs-- > 0) {
545       familyToCompression.put("Family2=asdads&!AASD", Compression.Algorithm.GZ);
546     }
547     if (numCfs-- > 0) {
548       familyToCompression.put("Family3", Compression.Algorithm.NONE);
549     }
550     return familyToCompression;
551   }
552   
553   /**
554    * Test that {@link HFileOutputFormat} RecordWriter uses compression settings
555    * from the column family descriptor
556    */
557   @Test
558   public void testColumnFamilyCompression() throws Exception {
559     Configuration conf = new Configuration(this.util.getConfiguration());
560     RecordWriter<ImmutableBytesWritable, KeyValue> writer = null;
561     TaskAttemptContext context = null;
562     Path dir =
563         util.getDataTestDir("testColumnFamilyCompression");
564 
565     HTable table = Mockito.mock(HTable.class);
566 
567     Map<String, Compression.Algorithm> configuredCompression =
568       new HashMap<String, Compression.Algorithm>();
569     Compression.Algorithm[] supportedAlgos = getSupportedCompressionAlgorithms();
570 
571     int familyIndex = 0;
572     for (byte[] family : FAMILIES) {
573       configuredCompression.put(Bytes.toString(family),
574                                 supportedAlgos[familyIndex++ % supportedAlgos.length]);
575     }
576     setupMockColumnFamilies(table, configuredCompression);
577 
578     // set up the table to return some mock keys
579     setupMockStartKeys(table);
580 
581     try {
582       // partial map red setup to get an operational writer for testing
583       // We turn off the sequence file compression, because DefaultCodec
584       // pollutes the GZip codec pool with an incompatible compressor.
585       conf.set("io.seqfile.compression.type", "NONE");
586       Job job = new Job(conf, "testLocalMRIncrementalLoad");
587       setupRandomGeneratorMapper(job);
588       HFileOutputFormat.configureIncrementalLoad(job, table);
589       FileOutputFormat.setOutputPath(job, dir);
590       context = getTestTaskAttemptContext(job);
591       HFileOutputFormat hof = new HFileOutputFormat();
592       writer = hof.getRecordWriter(context);
593 
594       // write out random rows
595       writeRandomKeyValues(writer, context, ROWSPERSPLIT);
596       writer.close(context);
597 
598       // Make sure that a directory was created for every CF
599       FileSystem fileSystem = dir.getFileSystem(conf);
600       
601       // commit so that the filesystem has one directory per column family
602       hof.getOutputCommitter(context).commitTask(context);
603       for (byte[] family : FAMILIES) {
604         String familyStr = new String(family);
605         boolean found = false;
606         for (FileStatus f : fileSystem.listStatus(dir)) {
607 
608           if (Bytes.toString(family).equals(f.getPath().getName())) {
609             // we found a matching directory
610             found = true;
611 
612             // verify that the compression on this file matches the configured
613             // compression
614             Path dataFilePath = fileSystem.listStatus(f.getPath())[0].getPath();
615             Reader reader = HFile.createReader(fileSystem, dataFilePath,
616                 new CacheConfig(conf));
617             reader.loadFileInfo();
618             assertEquals("Incorrect compression used for column family " + familyStr
619                          + "(reader: " + reader + ")",
620                          configuredCompression.get(familyStr), reader.getCompressionAlgorithm());
621             break;
622           }
623         }
624 
625         if (!found) {
626           fail("HFile for column family " + familyStr + " not found");
627         }
628       }
629 
630     } finally {
631       dir.getFileSystem(conf).delete(dir, true);
632     }
633   }
634 
635 
636   /**
637    * @return
638    */
639   private Compression.Algorithm[] getSupportedCompressionAlgorithms() {
640     String[] allAlgos = HFile.getSupportedCompressionAlgorithms();
641     List<Compression.Algorithm> supportedAlgos = Lists.newArrayList();
642 
643     for (String algoName : allAlgos) {
644       try {
645         Compression.Algorithm algo = Compression.getCompressionAlgorithmByName(algoName);
646         algo.getCompressor();
647         supportedAlgos.add(algo);
648       }catch (Exception e) {
649         // this algo is not available
650       }
651     }
652 
653     return supportedAlgos.toArray(new Compression.Algorithm[0]);
654   }
655 
656 
657   /**
658    * Write random values to the writer assuming a table created using
659    * {@link #FAMILIES} as column family descriptors
660    */
661   private void writeRandomKeyValues(RecordWriter<ImmutableBytesWritable, KeyValue> writer, TaskAttemptContext context,
662       int numRows)
663       throws IOException, InterruptedException {
664     byte keyBytes[] = new byte[Bytes.SIZEOF_INT];
665     int valLength = 10;
666     byte valBytes[] = new byte[valLength];
667 
668     int taskId = context.getTaskAttemptID().getTaskID().getId();
669     assert taskId < Byte.MAX_VALUE : "Unit tests dont support > 127 tasks!";
670 
671     Random random = new Random();
672     for (int i = 0; i < numRows; i++) {
673 
674       Bytes.putInt(keyBytes, 0, i);
675       random.nextBytes(valBytes);
676       ImmutableBytesWritable key = new ImmutableBytesWritable(keyBytes);
677 
678       for (byte[] family : TestHFileOutputFormat.FAMILIES) {
679         KeyValue kv = new KeyValue(keyBytes, family,
680             PerformanceEvaluation.QUALIFIER_NAME, valBytes);
681         writer.write(key, kv);
682       }
683     }
684   }
685   
686   public static void main(String args[]) throws Exception {
687     new TestHFileOutputFormat().manualTest(args);
688   }
689   
690   public void manualTest(String args[]) throws Exception {
691     Configuration conf = HBaseConfiguration.create();    
692     util = new HBaseTestingUtility(conf);
693     if ("newtable".equals(args[0])) {
694       byte[] tname = args[1].getBytes();
695       HTable table = util.createTable(tname, FAMILIES);
696       HBaseAdmin admin = new HBaseAdmin(conf);
697       admin.disableTable(tname);
698       byte[][] startKeys = generateRandomStartKeys(5);
699       util.createMultiRegions(conf, table, FAMILIES[0], startKeys);
700       admin.enableTable(tname);
701     } else if ("incremental".equals(args[0])) {
702       byte[] tname = args[1].getBytes();
703       HTable table = new HTable(conf, tname);
704       Path outDir = new Path("incremental-out");
705       runIncrementalPELoad(conf, table, outDir);
706     } else {
707       throw new RuntimeException(
708           "usage: TestHFileOutputFormat newtable | incremental");
709     }
710   }
711 }