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.concurrent.Callable;
37  import java.util.Random;
38  
39  import junit.framework.Assert;
40  import org.apache.commons.logging.Log;
41  import org.apache.commons.logging.LogFactory;
42  import org.apache.hadoop.conf.Configuration;
43  import org.apache.hadoop.fs.FileStatus;
44  import org.apache.hadoop.fs.FileSystem;
45  import org.apache.hadoop.fs.Path;
46  import org.apache.hadoop.hbase.*;
47  import org.apache.hadoop.hbase.client.HBaseAdmin;
48  import org.apache.hadoop.hbase.client.HTable;
49  import org.apache.hadoop.hbase.client.Put;
50  import org.apache.hadoop.hbase.client.Result;
51  import org.apache.hadoop.hbase.client.ResultScanner;
52  import org.apache.hadoop.hbase.client.Scan;
53  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
54  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
55  import org.apache.hadoop.hbase.io.hfile.Compression;
56  import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
57  import org.apache.hadoop.hbase.io.hfile.HFile;
58  import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
59  import org.apache.hadoop.hbase.regionserver.Store;
60  import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
61  import org.apache.hadoop.hbase.util.Bytes;
62  import org.apache.hadoop.hbase.util.FSUtils;
63  import org.apache.hadoop.hbase.util.Threads;
64  import org.apache.hadoop.hbase.util.Writables;
65  import org.apache.hadoop.io.NullWritable;
66  import org.apache.hadoop.mapreduce.Job;
67  import org.apache.hadoop.mapreduce.Mapper;
68  import org.apache.hadoop.mapreduce.RecordWriter;
69  import org.apache.hadoop.mapreduce.TaskAttemptContext;
70  import org.apache.hadoop.mapreduce.TaskAttemptID;
71  import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
72  import org.junit.Before;
73  import org.junit.Test;
74  import org.junit.experimental.categories.Category;
75  import org.mockito.Mockito;
76  
77  import com.google.common.collect.Lists;
78  
79  /**
80   * Simple test for {@link KeyValueSortReducer} and {@link HFileOutputFormat}.
81   * Sets up and runs a mapreduce job that writes hfile output.
82   * Creates a few inner classes to implement splits and an inputformat that
83   * emits keys and values like those of {@link PerformanceEvaluation}.
84   */
85  @Category(LargeTests.class)
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 1 directory down from the attempt directory 
276       // and is the only file, e.g.
277       // _attempt__0000_r_000000_0/b/1979617994050536795
278       FileSystem fs = FileSystem.get(conf);
279       Path attemptDirectory = hof.getDefaultWorkFile(context, "").getParent();
280       FileStatus[] sub1 = fs.listStatus(attemptDirectory);
281       FileStatus[] file = fs.listStatus(sub1[0].getPath());
282 
283       // open as HFile Reader and pull out TIMERANGE FileInfo.
284       HFile.Reader rd = HFile.createReader(fs, file[0].getPath(),
285           new CacheConfig(conf));
286       Map<byte[],byte[]> finfo = rd.loadFileInfo();
287       byte[] range = finfo.get("TIMERANGE".getBytes());
288       assertNotNull(range);
289 
290       // unmarshall and check values.
291       TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
292       Writables.copyWritable(range, timeRangeTracker);
293       LOG.info(timeRangeTracker.getMinimumTimestamp() +
294           "...." + timeRangeTracker.getMaximumTimestamp());
295       assertEquals(1000, timeRangeTracker.getMinimumTimestamp());
296       assertEquals(2000, timeRangeTracker.getMaximumTimestamp());
297       rd.close();
298     } finally {
299       if (writer != null && context != null) writer.close(context);
300       dir.getFileSystem(conf).delete(dir, true);
301     }
302   }
303 
304   /**
305    * Run small MR job.
306    */
307   @Test
308   public void testWritingPEData() throws Exception {
309     Configuration conf = util.getConfiguration();
310     Path testDir = util.getDataTestDir("testWritingPEData");
311     FileSystem fs = testDir.getFileSystem(conf);
312 
313     // Set down this value or we OOME in eclipse.
314     conf.setInt("io.sort.mb", 20);
315     // Write a few files.
316     conf.setLong(HConstants.HREGION_MAX_FILESIZE, 64 * 1024);
317 
318     Job job = new Job(conf, "testWritingPEData");
319     setupRandomGeneratorMapper(job);
320     // This partitioner doesn't work well for number keys but using it anyways
321     // just to demonstrate how to configure it.
322     byte[] startKey = new byte[RandomKVGeneratingMapper.KEYLEN_DEFAULT];
323     byte[] endKey = new byte[RandomKVGeneratingMapper.KEYLEN_DEFAULT];
324 
325     Arrays.fill(startKey, (byte)0);
326     Arrays.fill(endKey, (byte)0xff);
327 
328     job.setPartitionerClass(SimpleTotalOrderPartitioner.class);
329     // Set start and end rows for partitioner.
330     SimpleTotalOrderPartitioner.setStartKey(job.getConfiguration(), startKey);
331     SimpleTotalOrderPartitioner.setEndKey(job.getConfiguration(), endKey);
332     job.setReducerClass(KeyValueSortReducer.class);
333     job.setOutputFormatClass(HFileOutputFormat.class);
334     job.setNumReduceTasks(4);
335 
336     FileOutputFormat.setOutputPath(job, testDir);
337     assertTrue(job.waitForCompletion(false));
338     FileStatus [] files = fs.listStatus(testDir);
339     assertTrue(files.length > 0);
340   }
341 
342   @Test
343   public void testJobConfiguration() throws Exception {
344     Job job = new Job();
345     HTable table = Mockito.mock(HTable.class);
346     setupMockStartKeys(table);
347     HFileOutputFormat.configureIncrementalLoad(job, table);
348     assertEquals(job.getNumReduceTasks(), 4);
349   }
350 
351   private byte [][] generateRandomStartKeys(int numKeys) {
352     Random random = new Random();
353     byte[][] ret = new byte[numKeys][];
354     // first region start key is always empty
355     ret[0] = HConstants.EMPTY_BYTE_ARRAY;
356     for (int i = 1; i < numKeys; i++) {
357       ret[i] = PerformanceEvaluation.generateValue(random);
358     }
359     return ret;
360   }
361 
362   @Test
363   public void testMRIncrementalLoad() throws Exception {
364     doIncrementalLoadTest(false);
365   }
366 
367   @Test
368   public void testMRIncrementalLoadWithSplit() throws Exception {
369     doIncrementalLoadTest(true);
370   }
371 
372   private void doIncrementalLoadTest(
373       boolean shouldChangeRegions) throws Exception {
374     Configuration conf = util.getConfiguration();
375     Path testDir = util.getDataTestDir("testLocalMRIncrementalLoad");
376     byte[][] startKeys = generateRandomStartKeys(5);
377 
378     try {
379       util.startMiniCluster();
380       HBaseAdmin admin = new HBaseAdmin(conf);
381       HTable table = util.createTable(TABLE_NAME, FAMILIES);
382       assertEquals("Should start with empty table",
383           0, util.countRows(table));
384       int numRegions = util.createMultiRegions(
385           util.getConfiguration(), table, FAMILIES[0], startKeys);
386       assertEquals("Should make 5 regions", numRegions, 5);
387 
388       // Generate the bulk load files
389       util.startMiniMapReduceCluster();
390       runIncrementalPELoad(conf, table, testDir);
391       // This doesn't write into the table, just makes files
392       assertEquals("HFOF should not touch actual table",
393           0, util.countRows(table));
394 
395 
396       // Make sure that a directory was created for every CF
397       int dir = 0;
398       for (FileStatus f : testDir.getFileSystem(conf).listStatus(testDir)) {
399         for (byte[] family : FAMILIES) {
400           if (Bytes.toString(family).equals(f.getPath().getName())) {
401             ++dir;
402           }
403         }
404       }
405       assertEquals("Column family not found in FS.", FAMILIES.length, dir);
406 
407       // handle the split case
408       if (shouldChangeRegions) {
409         LOG.info("Changing regions in table");
410         admin.disableTable(table.getTableName());
411         while(util.getMiniHBaseCluster().getMaster().getAssignmentManager().
412             isRegionsInTransition()) {
413           Threads.sleep(200);
414           LOG.info("Waiting on table to finish disabling");
415         }
416         byte[][] newStartKeys = generateRandomStartKeys(15);
417         util.createMultiRegions(
418             util.getConfiguration(), table, FAMILIES[0], newStartKeys);
419         admin.enableTable(table.getTableName());
420         while (table.getRegionsInfo().size() != 15 ||
421             !admin.isTableAvailable(table.getTableName())) {
422           Thread.sleep(200);
423           LOG.info("Waiting for new region assignment to happen");
424         }
425       }
426 
427       // Perform the actual load
428       new LoadIncrementalHFiles(conf).doBulkLoad(testDir, table);
429 
430       // Ensure data shows up
431       int expectedRows = NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;
432       assertEquals("LoadIncrementalHFiles should put expected data in table",
433           expectedRows, util.countRows(table));
434       Scan scan = new Scan();
435       ResultScanner results = table.getScanner(scan);
436       int count = 0;
437       for (Result res : results) {
438         count++;
439         assertEquals(FAMILIES.length, res.raw().length);
440         KeyValue first = res.raw()[0];
441         for (KeyValue kv : res.raw()) {
442           assertTrue(KeyValue.COMPARATOR.matchingRows(first, kv));
443           assertTrue(Bytes.equals(first.getValue(), kv.getValue()));
444         }
445       }
446       results.close();
447       String tableDigestBefore = util.checksumRows(table);
448 
449       // Cause regions to reopen
450       admin.disableTable(TABLE_NAME);
451       while (!admin.isTableDisabled(TABLE_NAME)) {
452         Thread.sleep(200);
453         LOG.info("Waiting for table to disable");
454       }
455       admin.enableTable(TABLE_NAME);
456       util.waitTableAvailable(TABLE_NAME, 30000);
457       assertEquals("Data should remain after reopening of regions",
458           tableDigestBefore, util.checksumRows(table));
459     } finally {
460       util.shutdownMiniMapReduceCluster();
461       util.shutdownMiniCluster();
462     }
463   }
464 
465   private void runIncrementalPELoad(
466       Configuration conf, HTable table, Path outDir)
467   throws Exception {
468     Job job = new Job(conf, "testLocalMRIncrementalLoad");
469     setupRandomGeneratorMapper(job);
470     HFileOutputFormat.configureIncrementalLoad(job, table);
471     FileOutputFormat.setOutputPath(job, outDir);
472 
473     Assert.assertFalse( util.getTestFileSystem().exists(outDir)) ;
474 
475     assertEquals(table.getRegionsInfo().size(),
476       job.getNumReduceTasks());
477 
478     assertTrue(job.waitForCompletion(true));
479   }
480 
481   /**
482    * Test for
483    * {@link HFileOutputFormat#createFamilyCompressionMap(Configuration)}. Tests
484    * that the compression map is correctly deserialized from configuration
485    *
486    * @throws IOException
487    */
488   @Test
489   public void testCreateFamilyCompressionMap() throws IOException {
490     for (int numCfs = 0; numCfs <= 3; numCfs++) {
491       Configuration conf = new Configuration(this.util.getConfiguration());
492       Map<String, Compression.Algorithm> familyToCompression = getMockColumnFamilies(numCfs);
493       HTable table = Mockito.mock(HTable.class);
494       setupMockColumnFamilies(table, familyToCompression);
495       HFileOutputFormat.configureCompression(table, conf);
496 
497       // read back family specific compression setting from the configuration
498       Map<byte[], String> retrievedFamilyToCompressionMap = HFileOutputFormat.createFamilyCompressionMap(conf);
499 
500       // test that we have a value for all column families that matches with the
501       // used mock values
502       for (Entry<String, Algorithm> entry : familyToCompression.entrySet()) {
503         assertEquals("Compression configuration incorrect for column family:" + entry.getKey(), entry.getValue()
504                      .getName(), retrievedFamilyToCompressionMap.get(entry.getKey().getBytes()));
505       }
506     }
507   }
508 
509   private void setupMockColumnFamilies(HTable table,
510     Map<String, Compression.Algorithm> familyToCompression) throws IOException
511   {
512     HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME);
513     for (Entry<String, Compression.Algorithm> entry : familyToCompression.entrySet()) {
514       mockTableDescriptor.addFamily(new HColumnDescriptor(entry.getKey())
515           .setMaxVersions(1)
516           .setCompressionType(entry.getValue())
517           .setBlockCacheEnabled(false)
518           .setTimeToLive(0));
519     }
520     Mockito.doReturn(mockTableDescriptor).when(table).getTableDescriptor();
521   }
522 
523   private void setupMockStartKeys(HTable table) throws IOException {
524     byte[][] mockKeys = new byte[][] {
525         HConstants.EMPTY_BYTE_ARRAY,
526         Bytes.toBytes("aaa"),
527         Bytes.toBytes("ggg"),
528         Bytes.toBytes("zzz")
529     };
530     Mockito.doReturn(mockKeys).when(table).getStartKeys();
531   }
532 
533   /**
534    * @return a map from column family names to compression algorithms for
535    *         testing column family compression. Column family names have special characters
536    */
537   private Map<String, Compression.Algorithm> getMockColumnFamilies(int numCfs) {
538     Map<String, Compression.Algorithm> familyToCompression = new HashMap<String, Compression.Algorithm>();
539     // use column family names having special characters
540     if (numCfs-- > 0) {
541       familyToCompression.put("Family1!@#!@#&", Compression.Algorithm.LZO);
542     }
543     if (numCfs-- > 0) {
544       familyToCompression.put("Family2=asdads&!AASD", Compression.Algorithm.SNAPPY);
545     }
546     if (numCfs-- > 0) {
547       familyToCompression.put("Family2=asdads&!AASD", Compression.Algorithm.GZ);
548     }
549     if (numCfs-- > 0) {
550       familyToCompression.put("Family3", Compression.Algorithm.NONE);
551     }
552     return familyToCompression;
553   }
554 
555   /**
556    * Test that {@link HFileOutputFormat} RecordWriter uses compression settings
557    * from the column family descriptor
558    */
559   @Test
560   public void testColumnFamilyCompression() throws Exception {
561     Configuration conf = new Configuration(this.util.getConfiguration());
562     RecordWriter<ImmutableBytesWritable, KeyValue> writer = null;
563     TaskAttemptContext context = null;
564     Path dir =
565         util.getDataTestDir("testColumnFamilyCompression");
566 
567     HTable table = Mockito.mock(HTable.class);
568 
569     Map<String, Compression.Algorithm> configuredCompression =
570       new HashMap<String, Compression.Algorithm>();
571     Compression.Algorithm[] supportedAlgos = getSupportedCompressionAlgorithms();
572 
573     int familyIndex = 0;
574     for (byte[] family : FAMILIES) {
575       configuredCompression.put(Bytes.toString(family),
576                                 supportedAlgos[familyIndex++ % supportedAlgos.length]);
577     }
578     setupMockColumnFamilies(table, configuredCompression);
579 
580     // set up the table to return some mock keys
581     setupMockStartKeys(table);
582 
583     try {
584       // partial map red setup to get an operational writer for testing
585       // We turn off the sequence file compression, because DefaultCodec
586       // pollutes the GZip codec pool with an incompatible compressor.
587       conf.set("io.seqfile.compression.type", "NONE");
588       Job job = new Job(conf, "testLocalMRIncrementalLoad");
589       setupRandomGeneratorMapper(job);
590       HFileOutputFormat.configureIncrementalLoad(job, table);
591       FileOutputFormat.setOutputPath(job, dir);
592       context = getTestTaskAttemptContext(job);
593       HFileOutputFormat hof = new HFileOutputFormat();
594       writer = hof.getRecordWriter(context);
595 
596       // write out random rows
597       writeRandomKeyValues(writer, context, ROWSPERSPLIT);
598       writer.close(context);
599 
600       // Make sure that a directory was created for every CF
601       FileSystem fileSystem = dir.getFileSystem(conf);
602 
603       // commit so that the filesystem has one directory per column family
604       hof.getOutputCommitter(context).commitTask(context);
605       hof.getOutputCommitter(context).commitJob(context);
606       for (byte[] family : FAMILIES) {
607         String familyStr = new String(family);
608         boolean found = false;
609         for (FileStatus f : fileSystem.listStatus(dir)) {
610 
611           if (Bytes.toString(family).equals(f.getPath().getName())) {
612             // we found a matching directory
613             found = true;
614 
615             // verify that the compression on this file matches the configured
616             // compression
617             Path dataFilePath = fileSystem.listStatus(f.getPath())[0].getPath();
618             Reader reader = HFile.createReader(fileSystem, dataFilePath,
619                 new CacheConfig(conf));
620             reader.loadFileInfo();
621             assertEquals("Incorrect compression used for column family " + familyStr
622                          + "(reader: " + reader + ")",
623                          configuredCompression.get(familyStr), reader.getCompressionAlgorithm());
624             break;
625           }
626         }
627 
628         if (!found) {
629           fail("HFile for column family " + familyStr + " not found");
630         }
631       }
632 
633     } finally {
634       dir.getFileSystem(conf).delete(dir, true);
635     }
636   }
637 
638 
639   /**
640    * @return
641    */
642   private Compression.Algorithm[] getSupportedCompressionAlgorithms() {
643     String[] allAlgos = HFile.getSupportedCompressionAlgorithms();
644     List<Compression.Algorithm> supportedAlgos = Lists.newArrayList();
645 
646     for (String algoName : allAlgos) {
647       try {
648         Compression.Algorithm algo = Compression.getCompressionAlgorithmByName(algoName);
649         algo.getCompressor();
650         supportedAlgos.add(algo);
651       }catch (Exception e) {
652         // this algo is not available
653       }
654     }
655 
656     return supportedAlgos.toArray(new Compression.Algorithm[0]);
657   }
658 
659 
660   /**
661    * Write random values to the writer assuming a table created using
662    * {@link #FAMILIES} as column family descriptors
663    */
664   private void writeRandomKeyValues(RecordWriter<ImmutableBytesWritable, KeyValue> writer, TaskAttemptContext context,
665       int numRows)
666       throws IOException, InterruptedException {
667     byte keyBytes[] = new byte[Bytes.SIZEOF_INT];
668     int valLength = 10;
669     byte valBytes[] = new byte[valLength];
670 
671     int taskId = context.getTaskAttemptID().getTaskID().getId();
672     assert taskId < Byte.MAX_VALUE : "Unit tests dont support > 127 tasks!";
673 
674     Random random = new Random();
675     for (int i = 0; i < numRows; i++) {
676 
677       Bytes.putInt(keyBytes, 0, i);
678       random.nextBytes(valBytes);
679       ImmutableBytesWritable key = new ImmutableBytesWritable(keyBytes);
680 
681       for (byte[] family : TestHFileOutputFormat.FAMILIES) {
682         KeyValue kv = new KeyValue(keyBytes, family,
683             PerformanceEvaluation.QUALIFIER_NAME, valBytes);
684         writer.write(key, kv);
685       }
686     }
687   }
688 
689   /**
690    * This test is to test the scenario happened in HBASE-6901.
691    * All files are bulk loaded and excluded from minor compaction.
692    * Without the fix of HBASE-6901, an ArrayIndexOutOfBoundsException
693    * will be thrown.
694    */
695   @Test
696   public void testExcludeAllFromMinorCompaction() throws Exception {
697     Configuration conf = util.getConfiguration();
698     conf.setInt("hbase.hstore.compaction.min", 2);
699     generateRandomStartKeys(5);
700 
701     try {
702       util.startMiniCluster();
703       final FileSystem fs = util.getDFSCluster().getFileSystem();
704       HBaseAdmin admin = new HBaseAdmin(conf);
705       HTable table = util.createTable(TABLE_NAME, FAMILIES);
706       assertEquals("Should start with empty table", 0, util.countRows(table));
707 
708       // deep inspection: get the StoreFile dir
709       final Path storePath = Store.getStoreHomedir(
710           HTableDescriptor.getTableDir(FSUtils.getRootDir(conf), TABLE_NAME),
711           admin.getTableRegions(TABLE_NAME).get(0).getEncodedName(),
712           FAMILIES[0]);
713       assertEquals(0, fs.listStatus(storePath).length);
714 
715       // Generate two bulk load files
716       conf.setBoolean("hbase.mapreduce.hfileoutputformat.compaction.exclude",
717           true);
718       util.startMiniMapReduceCluster();
719 
720       for (int i = 0; i < 2; i++) {
721         Path testDir = util.getDataTestDir("testExcludeAllFromMinorCompaction_" + i);
722         runIncrementalPELoad(conf, table, testDir);
723         // Perform the actual load
724         new LoadIncrementalHFiles(conf).doBulkLoad(testDir, table);
725       }
726 
727       // Ensure data shows up
728       int expectedRows = 2 * NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;
729       assertEquals("LoadIncrementalHFiles should put expected data in table",
730           expectedRows, util.countRows(table));
731 
732       // should have a second StoreFile now
733       assertEquals(2, fs.listStatus(storePath).length);
734 
735       // minor compactions shouldn't get rid of the file
736       admin.compact(TABLE_NAME);
737       try {
738         quickPoll(new Callable<Boolean>() {
739           public Boolean call() throws Exception {
740             return fs.listStatus(storePath).length == 1;
741           }
742         }, 5000);
743         throw new IOException("SF# = " + fs.listStatus(storePath).length);
744       } catch (AssertionError ae) {
745         // this is expected behavior
746       }
747 
748       // a major compaction should work though
749       admin.majorCompact(TABLE_NAME);
750       quickPoll(new Callable<Boolean>() {
751         public Boolean call() throws Exception {
752           return fs.listStatus(storePath).length == 1;
753         }
754       }, 5000);
755 
756     } finally {
757       util.shutdownMiniMapReduceCluster();
758       util.shutdownMiniCluster();
759     }
760   }
761 
762   @Test
763   public void testExcludeMinorCompaction() throws Exception {
764     Configuration conf = util.getConfiguration();
765     conf.setInt("hbase.hstore.compaction.min", 2);
766     Path testDir = util.getDataTestDir("testExcludeMinorCompaction");
767     generateRandomStartKeys(5);
768 
769     try {
770       util.startMiniCluster();
771       final FileSystem fs = util.getDFSCluster().getFileSystem();
772       HBaseAdmin admin = new HBaseAdmin(conf);
773       HTable table = util.createTable(TABLE_NAME, FAMILIES);
774       assertEquals("Should start with empty table", 0, util.countRows(table));
775 
776       // deep inspection: get the StoreFile dir
777       final Path storePath = Store.getStoreHomedir(
778           HTableDescriptor.getTableDir(FSUtils.getRootDir(conf), TABLE_NAME),
779           admin.getTableRegions(TABLE_NAME).get(0).getEncodedName(),
780           FAMILIES[0]);
781       assertEquals(0, fs.listStatus(storePath).length);
782 
783       // put some data in it and flush to create a storefile
784       Put p = new Put(Bytes.toBytes("test"));
785       p.add(FAMILIES[0], Bytes.toBytes("1"), Bytes.toBytes("1"));
786       table.put(p);
787       admin.flush(TABLE_NAME);
788       assertEquals(1, util.countRows(table));
789       quickPoll(new Callable<Boolean>() {
790         public Boolean call() throws Exception {
791           return fs.listStatus(storePath).length == 1;
792         }
793       }, 5000);
794 
795       // Generate a bulk load file with more rows
796       conf.setBoolean("hbase.mapreduce.hfileoutputformat.compaction.exclude",
797           true);
798       util.startMiniMapReduceCluster();
799       runIncrementalPELoad(conf, table, testDir);
800 
801       // Perform the actual load
802       new LoadIncrementalHFiles(conf).doBulkLoad(testDir, table);
803 
804       // Ensure data shows up
805       int expectedRows = NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;
806       assertEquals("LoadIncrementalHFiles should put expected data in table",
807           expectedRows + 1, util.countRows(table));
808 
809       // should have a second StoreFile now
810       assertEquals(2, fs.listStatus(storePath).length);
811 
812       // minor compactions shouldn't get rid of the file
813       admin.compact(TABLE_NAME);
814       try {
815         quickPoll(new Callable<Boolean>() {
816           public Boolean call() throws Exception {
817             return fs.listStatus(storePath).length == 1;
818           }
819         }, 5000);
820         throw new IOException("SF# = " + fs.listStatus(storePath).length);
821       } catch (AssertionError ae) {
822         // this is expected behavior
823       }
824 
825       // a major compaction should work though
826       admin.majorCompact(TABLE_NAME);
827       quickPoll(new Callable<Boolean>() {
828         public Boolean call() throws Exception {
829           return fs.listStatus(storePath).length == 1;
830         }
831       }, 5000);
832 
833     } finally {
834       util.shutdownMiniMapReduceCluster();
835       util.shutdownMiniCluster();
836     }
837   }
838 
839   private void quickPoll(Callable<Boolean> c, int waitMs) throws Exception {
840     int sleepMs = 10;
841     int retries = (int) Math.ceil(((double) waitMs) / sleepMs);
842     while (retries-- > 0) {
843       if (c.call().booleanValue()) {
844         return;
845       }
846       Thread.sleep(sleepMs);
847     }
848     fail();
849   }
850 
851   public static void main(String args[]) throws Exception {
852     new TestHFileOutputFormat().manualTest(args);
853   }
854 
855   public void manualTest(String args[]) throws Exception {
856     Configuration conf = HBaseConfiguration.create();
857     util = new HBaseTestingUtility(conf);
858     if ("newtable".equals(args[0])) {
859       byte[] tname = args[1].getBytes();
860       HTable table = util.createTable(tname, FAMILIES);
861       HBaseAdmin admin = new HBaseAdmin(conf);
862       admin.disableTable(tname);
863       byte[][] startKeys = generateRandomStartKeys(5);
864       util.createMultiRegions(conf, table, FAMILIES[0], startKeys);
865       admin.enableTable(tname);
866     } else if ("incremental".equals(args[0])) {
867       byte[] tname = args[1].getBytes();
868       HTable table = new HTable(conf, tname);
869       Path outDir = new Path("incremental-out");
870       runIncrementalPELoad(conf, table, outDir);
871     } else {
872       throw new RuntimeException(
873           "usage: TestHFileOutputFormat newtable | incremental");
874     }
875   }
876 
877   @org.junit.Rule
878   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
879     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
880 }
881