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