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.Map;
34  import java.util.Map.Entry;
35  import java.util.Set;
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.encoding.DataBlockEncoding;
55  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
56  import org.apache.hadoop.hbase.io.hfile.Compression;
57  import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
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.regionserver.StoreFile.BloomType;
65  import org.apache.hadoop.hbase.util.Bytes;
66  import org.apache.hadoop.hbase.util.FSUtils;
67  import org.apache.hadoop.hbase.util.Threads;
68  import org.apache.hadoop.hbase.util.Writables;
69  import org.apache.hadoop.io.NullWritable;
70  import org.apache.hadoop.mapreduce.Job;
71  import org.apache.hadoop.mapreduce.Mapper;
72  import org.apache.hadoop.mapreduce.RecordWriter;
73  import org.apache.hadoop.mapreduce.TaskAttemptContext;
74  import org.apache.hadoop.mapreduce.TaskAttemptID;
75  import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
76  import org.junit.Before;
77  import org.junit.Test;
78  import org.junit.experimental.categories.Category;
79  import org.mockito.Mockito;
80  
81  /**
82   * Simple test for {@link KeyValueSortReducer} and {@link HFileOutputFormat}.
83   * Sets up and runs a mapreduce job that writes hfile output.
84   * Creates a few inner classes to implement splits and an inputformat that
85   * emits keys and values like those of {@link PerformanceEvaluation}.
86   */
87  @Category(LargeTests.class)
88  public class TestHFileOutputFormat  {
89    private final static int ROWSPERSPLIT = 1024;
90  
91    private static final byte[][] FAMILIES
92      = { Bytes.add(PerformanceEvaluation.FAMILY_NAME, Bytes.toBytes("-A"))
93        , Bytes.add(PerformanceEvaluation.FAMILY_NAME, Bytes.toBytes("-B"))};
94    private static final byte[] TABLE_NAME = Bytes.toBytes("TestTable");
95  
96    private HBaseTestingUtility util = new HBaseTestingUtility();
97  
98    private static Log LOG = LogFactory.getLog(TestHFileOutputFormat.class);
99  
100   /**
101    * Simple mapper that makes KeyValue output.
102    */
103   static class RandomKVGeneratingMapper
104   extends Mapper<NullWritable, NullWritable,
105                  ImmutableBytesWritable, KeyValue> {
106 
107     private int keyLength;
108     private static final int KEYLEN_DEFAULT=10;
109     private static final String KEYLEN_CONF="randomkv.key.length";
110 
111     private int valLength;
112     private static final int VALLEN_DEFAULT=10;
113     private static final String VALLEN_CONF="randomkv.val.length";
114 
115     @Override
116     protected void setup(Context context) throws IOException,
117         InterruptedException {
118       super.setup(context);
119 
120       Configuration conf = context.getConfiguration();
121       keyLength = conf.getInt(KEYLEN_CONF, KEYLEN_DEFAULT);
122       valLength = conf.getInt(VALLEN_CONF, VALLEN_DEFAULT);
123     }
124 
125     protected void map(
126         NullWritable n1, NullWritable n2,
127         Mapper<NullWritable, NullWritable,
128                ImmutableBytesWritable,KeyValue>.Context context)
129         throws java.io.IOException ,InterruptedException
130     {
131 
132       byte keyBytes[] = new byte[keyLength];
133       byte valBytes[] = new byte[valLength];
134 
135       int taskId = context.getTaskAttemptID().getTaskID().getId();
136       assert taskId < Byte.MAX_VALUE : "Unit tests dont support > 127 tasks!";
137 
138       Random random = new Random();
139       for (int i = 0; i < ROWSPERSPLIT; i++) {
140 
141         random.nextBytes(keyBytes);
142         // Ensure that unique tasks generate unique keys
143         keyBytes[keyLength - 1] = (byte)(taskId & 0xFF);
144         random.nextBytes(valBytes);
145         ImmutableBytesWritable key = new ImmutableBytesWritable(keyBytes);
146 
147         for (byte[] family : TestHFileOutputFormat.FAMILIES) {
148           KeyValue kv = new KeyValue(keyBytes, family,
149               PerformanceEvaluation.QUALIFIER_NAME, valBytes);
150           context.write(key, kv);
151         }
152       }
153     }
154   }
155 
156   @Before
157   public void cleanupDir() throws IOException {
158     util.cleanupTestDir();
159   }
160 
161 
162   private void setupRandomGeneratorMapper(Job job) {
163     job.setInputFormatClass(NMapInputFormat.class);
164     job.setMapperClass(RandomKVGeneratingMapper.class);
165     job.setMapOutputKeyClass(ImmutableBytesWritable.class);
166     job.setMapOutputValueClass(KeyValue.class);
167   }
168 
169   /**
170    * Test that {@link HFileOutputFormat} RecordWriter amends timestamps if
171    * passed a keyvalue whose timestamp is {@link HConstants#LATEST_TIMESTAMP}.
172    * @see <a href="https://issues.apache.org/jira/browse/HBASE-2615">HBASE-2615</a>
173    */
174   @Test
175   public void test_LATEST_TIMESTAMP_isReplaced()
176   throws Exception {
177     Configuration conf = new Configuration(this.util.getConfiguration());
178     RecordWriter<ImmutableBytesWritable, KeyValue> writer = null;
179     TaskAttemptContext context = null;
180     Path dir =
181       util.getDataTestDir("test_LATEST_TIMESTAMP_isReplaced");
182     try {
183       Job job = new Job(conf);
184       FileOutputFormat.setOutputPath(job, dir);
185       context = getTestTaskAttemptContext(job);
186       HFileOutputFormat hof = new HFileOutputFormat();
187       writer = hof.getRecordWriter(context);
188       final byte [] b = Bytes.toBytes("b");
189 
190       // Test 1.  Pass a KV that has a ts of LATEST_TIMESTAMP.  It should be
191       // changed by call to write.  Check all in kv is same but ts.
192       KeyValue kv = new KeyValue(b, b, b);
193       KeyValue original = kv.clone();
194       writer.write(new ImmutableBytesWritable(), kv);
195       assertFalse(original.equals(kv));
196       assertTrue(Bytes.equals(original.getRow(), kv.getRow()));
197       assertTrue(original.matchingColumn(kv.getFamily(), kv.getQualifier()));
198       assertNotSame(original.getTimestamp(), kv.getTimestamp());
199       assertNotSame(HConstants.LATEST_TIMESTAMP, kv.getTimestamp());
200 
201       // Test 2. Now test passing a kv that has explicit ts.  It should not be
202       // changed by call to record write.
203       kv = new KeyValue(b, b, b, kv.getTimestamp() - 1, b);
204       original = kv.clone();
205       writer.write(new ImmutableBytesWritable(), kv);
206       assertTrue(original.equals(kv));
207     } finally {
208       if (writer != null && context != null) writer.close(context);
209       dir.getFileSystem(conf).delete(dir, true);
210     }
211   }
212 
213   /**
214    * @return True if the available mapreduce is post-0.20.
215    */
216   private static boolean isPost020MapReduce() {
217     // Here is a coarse test for post 0.20 hadoop; TAC became an interface.
218     return TaskAttemptContext.class.isInterface();
219   }
220 
221   private TaskAttemptContext getTestTaskAttemptContext(final Job job)
222   throws IOException, Exception {
223     TaskAttemptContext context;
224     if (isPost020MapReduce()) {
225       TaskAttemptID id =
226         TaskAttemptID.forName("attempt_200707121733_0001_m_000000_0");
227       Class<?> clazz =
228         Class.forName("org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl");
229       Constructor<?> c = clazz.
230           getConstructor(Configuration.class, TaskAttemptID.class);
231       context = (TaskAttemptContext)c.newInstance(job.getConfiguration(), id);
232     } else {
233       context = org.apache.hadoop.hbase.mapreduce.hadoopbackport.InputSampler.
234         getTaskAttemptContext(job);
235     }
236     return context;
237   }
238 
239   /*
240    * Test that {@link HFileOutputFormat} creates an HFile with TIMERANGE
241    * metadata used by time-restricted scans.
242    */
243   @Test
244   public void test_TIMERANGE() throws Exception {
245     Configuration conf = new Configuration(this.util.getConfiguration());
246     RecordWriter<ImmutableBytesWritable, KeyValue> writer = null;
247     TaskAttemptContext context = null;
248     Path dir =
249       util.getDataTestDir("test_TIMERANGE_present");
250     LOG.info("Timerange dir writing to dir: "+ dir);
251     try {
252       // build a record writer using HFileOutputFormat
253       Job job = new Job(conf);
254       FileOutputFormat.setOutputPath(job, dir);
255       context = getTestTaskAttemptContext(job);
256       HFileOutputFormat hof = new HFileOutputFormat();
257       writer = hof.getRecordWriter(context);
258 
259       // Pass two key values with explicit times stamps
260       final byte [] b = Bytes.toBytes("b");
261 
262       // value 1 with timestamp 2000
263       KeyValue kv = new KeyValue(b, b, b, 2000, b);
264       KeyValue original = kv.clone();
265       writer.write(new ImmutableBytesWritable(), kv);
266       assertEquals(original,kv);
267 
268       // value 2 with timestamp 1000
269       kv = new KeyValue(b, b, b, 1000, b);
270       original = kv.clone();
271       writer.write(new ImmutableBytesWritable(), kv);
272       assertEquals(original, kv);
273 
274       // verify that the file has the proper FileInfo.
275       writer.close(context);
276 
277       // the generated file lives 1 directory down from the attempt directory
278       // and is the only file, e.g.
279       // _attempt__0000_r_000000_0/b/1979617994050536795
280       FileSystem fs = FileSystem.get(conf);
281       Path attemptDirectory = hof.getDefaultWorkFile(context, "").getParent();
282       FileStatus[] sub1 = fs.listStatus(attemptDirectory);
283       FileStatus[] file = fs.listStatus(sub1[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       rd.close();
300     } finally {
301       if (writer != null && context != null) writer.close(context);
302       dir.getFileSystem(conf).delete(dir, true);
303     }
304   }
305 
306   /**
307    * Run small MR job.
308    */
309   @Test
310   public void testWritingPEData() throws Exception {
311     Configuration conf = util.getConfiguration();
312     Path testDir = util.getDataTestDir("testWritingPEData");
313     FileSystem fs = testDir.getFileSystem(conf);
314 
315     // Set down this value or we OOME in eclipse.
316     conf.setInt("io.sort.mb", 20);
317     // Write a few files.
318     conf.setLong(HConstants.HREGION_MAX_FILESIZE, 64 * 1024);
319 
320     Job job = new Job(conf, "testWritingPEData");
321     setupRandomGeneratorMapper(job);
322     // This partitioner doesn't work well for number keys but using it anyways
323     // just to demonstrate how to configure it.
324     byte[] startKey = new byte[RandomKVGeneratingMapper.KEYLEN_DEFAULT];
325     byte[] endKey = new byte[RandomKVGeneratingMapper.KEYLEN_DEFAULT];
326 
327     Arrays.fill(startKey, (byte)0);
328     Arrays.fill(endKey, (byte)0xff);
329 
330     job.setPartitionerClass(SimpleTotalOrderPartitioner.class);
331     // Set start and end rows for partitioner.
332     SimpleTotalOrderPartitioner.setStartKey(job.getConfiguration(), startKey);
333     SimpleTotalOrderPartitioner.setEndKey(job.getConfiguration(), endKey);
334     job.setReducerClass(KeyValueSortReducer.class);
335     job.setOutputFormatClass(HFileOutputFormat.class);
336     job.setNumReduceTasks(4);
337 
338     FileOutputFormat.setOutputPath(job, testDir);
339     assertTrue(job.waitForCompletion(false));
340     FileStatus [] files = fs.listStatus(testDir);
341     assertTrue(files.length > 0);
342   }
343 
344   @Test
345   public void testJobConfiguration() throws Exception {
346     Job job = new Job(util.getConfiguration());
347     job.setWorkingDirectory(util.getDataTestDir("testJobConfiguration"));
348     HTable table = Mockito.mock(HTable.class);
349     setupMockStartKeys(table);
350     HFileOutputFormat.configureIncrementalLoad(job, table);
351     assertEquals(job.getNumReduceTasks(), 4);
352   }
353 
354   private byte [][] generateRandomStartKeys(int numKeys) {
355     Random random = new Random();
356     byte[][] ret = new byte[numKeys][];
357     // first region start key is always empty
358     ret[0] = HConstants.EMPTY_BYTE_ARRAY;
359     for (int i = 1; i < numKeys; i++) {
360       ret[i] = PerformanceEvaluation.generateValue(random);
361     }
362     return ret;
363   }
364 
365   @Test
366   public void testMRIncrementalLoad() throws Exception {
367     doIncrementalLoadTest(false);
368   }
369 
370   @Test
371   public void testMRIncrementalLoadWithSplit() throws Exception {
372     doIncrementalLoadTest(true);
373   }
374 
375   private void doIncrementalLoadTest(
376       boolean shouldChangeRegions) throws Exception {
377     Configuration conf = util.getConfiguration();
378     Path testDir = util.getDataTestDir("testLocalMRIncrementalLoad");
379     byte[][] startKeys = generateRandomStartKeys(5);
380 
381     try {
382       util.startMiniCluster();
383       HBaseAdmin admin = new HBaseAdmin(conf);
384       HTable table = util.createTable(TABLE_NAME, FAMILIES);
385       assertEquals("Should start with empty table",
386           0, util.countRows(table));
387       int numRegions = util.createMultiRegions(
388           util.getConfiguration(), table, FAMILIES[0], startKeys);
389       assertEquals("Should make 5 regions", numRegions, 5);
390 
391       // Generate the bulk load files
392       util.startMiniMapReduceCluster();
393       runIncrementalPELoad(conf, table, testDir);
394       // This doesn't write into the table, just makes files
395       assertEquals("HFOF should not touch actual table",
396           0, util.countRows(table));
397 
398 
399       // Make sure that a directory was created for every CF
400       int dir = 0;
401       for (FileStatus f : testDir.getFileSystem(conf).listStatus(testDir)) {
402         for (byte[] family : FAMILIES) {
403           if (Bytes.toString(family).equals(f.getPath().getName())) {
404             ++dir;
405           }
406         }
407       }
408       assertEquals("Column family not found in FS.", FAMILIES.length, dir);
409 
410       // handle the split case
411       if (shouldChangeRegions) {
412         LOG.info("Changing regions in table");
413         admin.disableTable(table.getTableName());
414         while(util.getMiniHBaseCluster().getMaster().getAssignmentManager().
415             isRegionsInTransition()) {
416           Threads.sleep(200);
417           LOG.info("Waiting on table to finish disabling");
418         }
419         byte[][] newStartKeys = generateRandomStartKeys(15);
420         util.createMultiRegions(
421             util.getConfiguration(), table, FAMILIES[0], newStartKeys);
422         admin.enableTable(table.getTableName());
423         while (table.getRegionsInfo().size() != 15 ||
424             !admin.isTableAvailable(table.getTableName())) {
425           Thread.sleep(200);
426           LOG.info("Waiting for new region assignment to happen");
427         }
428       }
429 
430       // Perform the actual load
431       new LoadIncrementalHFiles(conf).doBulkLoad(testDir, table);
432 
433       // Ensure data shows up
434       int expectedRows = NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;
435       assertEquals("LoadIncrementalHFiles should put expected data in table",
436           expectedRows, util.countRows(table));
437       Scan scan = new Scan();
438       ResultScanner results = table.getScanner(scan);
439       int count = 0;
440       for (Result res : results) {
441         count++;
442         assertEquals(FAMILIES.length, res.raw().length);
443         KeyValue first = res.raw()[0];
444         for (KeyValue kv : res.raw()) {
445           assertTrue(KeyValue.COMPARATOR.matchingRows(first, kv));
446           assertTrue(Bytes.equals(first.getValue(), kv.getValue()));
447         }
448       }
449       results.close();
450       String tableDigestBefore = util.checksumRows(table);
451 
452       // Cause regions to reopen
453       admin.disableTable(TABLE_NAME);
454       while (!admin.isTableDisabled(TABLE_NAME)) {
455         Thread.sleep(200);
456         LOG.info("Waiting for table to disable");
457       }
458       admin.enableTable(TABLE_NAME);
459       util.waitTableAvailable(TABLE_NAME, 30000);
460       assertEquals("Data should remain after reopening of regions",
461           tableDigestBefore, util.checksumRows(table));
462     } finally {
463       util.shutdownMiniMapReduceCluster();
464       util.shutdownMiniCluster();
465     }
466   }
467 
468   private void runIncrementalPELoad(
469       Configuration conf, HTable table, Path outDir)
470   throws Exception {
471     Job job = new Job(conf, "testLocalMRIncrementalLoad");
472     job.setWorkingDirectory(util.getDataTestDir("runIncrementalPELoad"));
473     setupRandomGeneratorMapper(job);
474     HFileOutputFormat.configureIncrementalLoad(job, table);
475     FileOutputFormat.setOutputPath(job, outDir);
476 
477     Assert.assertFalse( util.getTestFileSystem().exists(outDir)) ;
478 
479     assertEquals(table.getRegionsInfo().size(),
480       job.getNumReduceTasks());
481 
482     assertTrue(job.waitForCompletion(true));
483   }
484 
485   /**
486    * Test for {@link HFileOutputFormat#configureCompression(HTable,
487    * Configuration)} and {@link HFileOutputFormat#createFamilyCompressionMap
488    * (Configuration)}.
489    * Tests that the compression map is correctly serialized into
490    * and deserialized from configuration
491    *
492    * @throws IOException
493    */
494   @Test
495   public void testSerializeDeserializeFamilyCompressionMap() throws IOException {
496     for (int numCfs = 0; numCfs <= 3; numCfs++) {
497       Configuration conf = new Configuration(this.util.getConfiguration());
498       Map<String, Compression.Algorithm> familyToCompression =
499           getMockColumnFamiliesForCompression(numCfs);
500       HTable table = Mockito.mock(HTable.class);
501       setupMockColumnFamiliesForCompression(table, familyToCompression);
502       HFileOutputFormat.configureCompression(table, conf);
503 
504       // read back family specific compression setting from the configuration
505       Map<byte[], Algorithm> retrievedFamilyToCompressionMap = HFileOutputFormat
506           .createFamilyCompressionMap(conf);
507 
508       // test that we have a value for all column families that matches with the
509       // used mock values
510       for (Entry<String, Algorithm> entry : familyToCompression.entrySet()) {
511         assertEquals("Compression configuration incorrect for column family:"
512             + entry.getKey(), entry.getValue(),
513             retrievedFamilyToCompressionMap.get(entry.getKey().getBytes()));
514       }
515     }
516   }
517 
518   private void setupMockColumnFamiliesForCompression(HTable table,
519       Map<String, Compression.Algorithm> familyToCompression) throws IOException {
520     HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME);
521     for (Entry<String, Compression.Algorithm> entry : familyToCompression.entrySet()) {
522       mockTableDescriptor.addFamily(new HColumnDescriptor(entry.getKey())
523           .setMaxVersions(1)
524           .setCompressionType(entry.getValue())
525           .setBlockCacheEnabled(false)
526           .setTimeToLive(0));
527     }
528     Mockito.doReturn(mockTableDescriptor).when(table).getTableDescriptor();
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> getMockColumnFamiliesForCompression (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 for {@link HFileOutputFormat#configureBloomType(HTable,
555    * Configuration)} and {@link HFileOutputFormat#createFamilyBloomTypeMap
556    * (Configuration)}.
557    * Tests that the compression map is correctly serialized into
558    * and deserialized from configuration
559    *
560    * @throws IOException
561    */
562   @Test
563   public void testSerializeDeserializeFamilyBloomTypeMap() throws IOException {
564     for (int numCfs = 0; numCfs <= 2; numCfs++) {
565       Configuration conf = new Configuration(this.util.getConfiguration());
566       Map<String, BloomType> familyToBloomType =
567           getMockColumnFamiliesForBloomType(numCfs);
568       HTable table = Mockito.mock(HTable.class);
569       setupMockColumnFamiliesForBloomType(table,
570           familyToBloomType);
571       HFileOutputFormat.configureBloomType(table, conf);
572 
573       // read back family specific bloom type settings from the configuration
574       Map<byte[], BloomType> retrievedFamilyToBloomTypeMap =
575           HFileOutputFormat
576               .createFamilyBloomTypeMap(conf);
577 
578       // test that we have a value for all column families that matches with the
579       // used mock values
580       for (Entry<String, BloomType> entry : familyToBloomType.entrySet()) {
581         assertEquals("BloomType configuration incorrect for column family:" 
582             + entry.getKey(), entry.getValue(), 
583             retrievedFamilyToBloomTypeMap.get(entry.getKey().getBytes()));
584       }
585     }
586   }
587 
588   private void setupMockColumnFamiliesForBloomType(HTable table,
589       Map<String, BloomType> familyToDataBlockEncoding) throws IOException {
590     HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME);
591     for (Entry<String, BloomType> entry : familyToDataBlockEncoding.entrySet()) {
592       mockTableDescriptor.addFamily(new HColumnDescriptor(entry.getKey())
593           .setMaxVersions(1)
594           .setBloomFilterType(entry.getValue())
595           .setBlockCacheEnabled(false)
596           .setTimeToLive(0));
597     }
598     Mockito.doReturn(mockTableDescriptor).when(table).getTableDescriptor();
599   }
600 
601   /**
602    * @return a map from column family names to compression algorithms for
603    *         testing column family compression. Column family names have special characters
604    */
605   private Map<String, BloomType>
606   getMockColumnFamiliesForBloomType (int numCfs) {
607     Map<String, BloomType> familyToBloomType =
608         new HashMap<String, BloomType>();
609     // use column family names having special characters
610     if (numCfs-- > 0) {
611       familyToBloomType.put("Family1!@#!@#&", BloomType.ROW);
612     }
613     if (numCfs-- > 0) {
614       familyToBloomType.put("Family2=asdads&!AASD",
615           BloomType.ROWCOL);
616     }
617     if (numCfs-- > 0) {
618       familyToBloomType.put("Family3", BloomType.NONE);
619     }
620     return familyToBloomType;
621   }
622 
623   /**
624    * Test for {@link HFileOutputFormat#configureBlockSize(HTable,
625    * Configuration)} and {@link HFileOutputFormat#createFamilyBlockSizeMap
626    * (Configuration)}.
627    * Tests that the compression map is correctly serialized into
628    * and deserialized from configuration
629    *
630    * @throws IOException
631    */
632   @Test
633   public void testSerializeDeserializeFamilyBlockSizeMap() throws IOException {
634     for (int numCfs = 0; numCfs <= 3; numCfs++) {
635       Configuration conf = new Configuration(this.util.getConfiguration());
636       Map<String, Integer> familyToBlockSize =
637           getMockColumnFamiliesForBlockSize(numCfs);
638       HTable table = Mockito.mock(HTable.class);
639       setupMockColumnFamiliesForBlockSize(table,
640           familyToBlockSize);
641       HFileOutputFormat.configureBlockSize(table, conf);
642 
643       // read back family specific data block size from the configuration
644       Map<byte[], Integer> retrievedFamilyToBlockSizeMap =
645           HFileOutputFormat
646               .createFamilyBlockSizeMap(conf);
647 
648       // test that we have a value for all column families that matches with the
649       // used mock values
650       for (Entry<String, Integer> entry : familyToBlockSize.entrySet()) {
651         assertEquals("BlockSize configuration incorrect for column family:"
652             + entry.getKey(), entry.getValue(), 
653             retrievedFamilyToBlockSizeMap.get(entry.getKey().getBytes()));
654       }
655     }
656   }
657 
658   private void setupMockColumnFamiliesForBlockSize(HTable table,
659       Map<String, Integer> familyToDataBlockEncoding) throws IOException {
660     HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME);
661     for (Entry<String, Integer> entry : familyToDataBlockEncoding.entrySet()) {
662       mockTableDescriptor.addFamily(new HColumnDescriptor(entry.getKey())
663           .setMaxVersions(1)
664           .setBlocksize(entry.getValue())
665           .setBlockCacheEnabled(false)
666           .setTimeToLive(0));
667     }
668     Mockito.doReturn(mockTableDescriptor).when(table).getTableDescriptor();
669   }
670 
671   /**
672    * @return a map from column family names to compression algorithms for
673    *         testing column family compression. Column family names have special characters
674    */
675   private Map<String, Integer>
676   getMockColumnFamiliesForBlockSize (int numCfs) {
677     Map<String, Integer> familyToBlockSize =
678         new HashMap<String, Integer>();
679     // use column family names having special characters
680     if (numCfs-- > 0) {
681       familyToBlockSize.put("Family1!@#!@#&", 1234);
682     }
683     if (numCfs-- > 0) {
684       familyToBlockSize.put("Family2=asdads&!AASD",
685           Integer.MAX_VALUE);
686     }
687     if (numCfs-- > 0) {
688       familyToBlockSize.put("Family2=asdads&!AASD",
689           Integer.MAX_VALUE);
690     }
691     if (numCfs-- > 0) {
692       familyToBlockSize.put("Family3", 0);
693     }
694     return familyToBlockSize;
695   }
696 
697     /**
698    * Test for {@link HFileOutputFormat#configureDataBlockEncoding(HTable,
699    * Configuration)} and {@link HFileOutputFormat#createFamilyDataBlockEncodingMap
700    * (Configuration)}.
701    * Tests that the compression map is correctly serialized into
702    * and deserialized from configuration
703    *
704    * @throws IOException
705    */
706   @Test
707   public void testSerializeDeserializeFamilyDataBlockEncodingMap() throws IOException {
708     for (int numCfs = 0; numCfs <= 3; numCfs++) {
709       Configuration conf = new Configuration(this.util.getConfiguration());
710       Map<String, DataBlockEncoding> familyToDataBlockEncoding =
711           getMockColumnFamiliesForDataBlockEncoding(numCfs);
712       HTable table = Mockito.mock(HTable.class);
713       setupMockColumnFamiliesForDataBlockEncoding(table,
714           familyToDataBlockEncoding);
715       HFileOutputFormat.configureDataBlockEncoding(table, conf);
716 
717       // read back family specific data block encoding settings from the configuration
718       Map<byte[], HFileDataBlockEncoder> retrievedFamilyToDataBlockEncodingMap =
719           HFileOutputFormat
720               .createFamilyDataBlockEncodingMap(conf);
721 
722       // test that we have a value for all column families that matches with the
723       // used mock values
724       for (Entry<String, DataBlockEncoding> entry : familyToDataBlockEncoding.entrySet()) {
725         assertEquals("DataBlockEncoding configuration incorrect for column family:"
726             + entry.getKey(), entry.getValue(),
727             retrievedFamilyToDataBlockEncodingMap.get(entry.getKey().getBytes
728                 ()).getEncodingOnDisk());
729       }
730     }
731   }
732 
733   private void setupMockColumnFamiliesForDataBlockEncoding(HTable table,
734       Map<String, DataBlockEncoding> familyToDataBlockEncoding) throws IOException {
735     HTableDescriptor mockTableDescriptor = new HTableDescriptor(TABLE_NAME);
736     for (Entry<String, DataBlockEncoding> entry : familyToDataBlockEncoding.entrySet()) {
737       mockTableDescriptor.addFamily(new HColumnDescriptor(entry.getKey())
738           .setMaxVersions(1)
739           .setDataBlockEncoding(entry.getValue())
740           .setBlockCacheEnabled(false)
741           .setTimeToLive(0));
742     }
743     Mockito.doReturn(mockTableDescriptor).when(table).getTableDescriptor();
744   }
745 
746   /**
747    * @return a map from column family names to compression algorithms for
748    *         testing column family compression. Column family names have special characters
749    */
750   private Map<String, DataBlockEncoding>
751       getMockColumnFamiliesForDataBlockEncoding (int numCfs) {
752     Map<String, DataBlockEncoding> familyToDataBlockEncoding =
753         new HashMap<String, DataBlockEncoding>();
754     // use column family names having special characters
755     if (numCfs-- > 0) {
756       familyToDataBlockEncoding.put("Family1!@#!@#&", DataBlockEncoding.DIFF);
757     }
758     if (numCfs-- > 0) {
759       familyToDataBlockEncoding.put("Family2=asdads&!AASD",
760           DataBlockEncoding.FAST_DIFF);
761     }
762     if (numCfs-- > 0) {
763       familyToDataBlockEncoding.put("Family2=asdads&!AASD",
764           DataBlockEncoding.PREFIX);
765     }
766     if (numCfs-- > 0) {
767       familyToDataBlockEncoding.put("Family3", DataBlockEncoding.NONE);
768     }
769     return familyToDataBlockEncoding;
770   }
771 
772   private void setupMockStartKeys(HTable table) throws IOException {
773     byte[][] mockKeys = new byte[][] {
774         HConstants.EMPTY_BYTE_ARRAY,
775         Bytes.toBytes("aaa"),
776         Bytes.toBytes("ggg"),
777         Bytes.toBytes("zzz")
778     };
779     Mockito.doReturn(mockKeys).when(table).getStartKeys();
780   }
781 
782   /**
783    * Test that {@link HFileOutputFormat} RecordWriter uses compression and
784    * bloom filter settings from the column family descriptor
785    */
786   @Test
787   public void testColumnFamilySettings() throws Exception {
788     Configuration conf = new Configuration(this.util.getConfiguration());
789     RecordWriter<ImmutableBytesWritable, KeyValue> writer = null;
790     TaskAttemptContext context = null;
791     Path dir = util.getDataTestDir("testColumnFamilySettings");
792 
793     // Setup table descriptor
794     HTable table = Mockito.mock(HTable.class);
795     HTableDescriptor htd = new HTableDescriptor(TABLE_NAME);
796     Mockito.doReturn(htd).when(table).getTableDescriptor();
797     for (HColumnDescriptor hcd: this.util.generateColumnDescriptors()) {
798       htd.addFamily(hcd);
799     }
800 
801     // set up the table to return some mock keys
802     setupMockStartKeys(table);
803 
804     try {
805       // partial map red setup to get an operational writer for testing
806       // We turn off the sequence file compression, because DefaultCodec
807       // pollutes the GZip codec pool with an incompatible compressor.
808       conf.set("io.seqfile.compression.type", "NONE");
809       Job job = new Job(conf, "testLocalMRIncrementalLoad");
810       job.setWorkingDirectory(util.getDataTestDir("testColumnFamilyCompression"));
811       setupRandomGeneratorMapper(job);
812       HFileOutputFormat.configureIncrementalLoad(job, table);
813       FileOutputFormat.setOutputPath(job, dir);
814       context = getTestTaskAttemptContext(job);
815       HFileOutputFormat hof = new HFileOutputFormat();
816       writer = hof.getRecordWriter(context);
817 
818       // write out random rows
819       writeRandomKeyValues(writer, context, htd.getFamiliesKeys(), ROWSPERSPLIT);
820       writer.close(context);
821 
822       // Make sure that a directory was created for every CF
823       FileSystem fs = dir.getFileSystem(conf);
824 
825       // commit so that the filesystem has one directory per column family
826       hof.getOutputCommitter(context).commitTask(context);
827       hof.getOutputCommitter(context).commitJob(context);
828       FileStatus[] families = FSUtils.listStatus(fs, dir, new FSUtils.FamilyDirFilter(fs));
829       assertEquals(htd.getFamilies().size(), families.length);
830       for (FileStatus f : families) {
831         String familyStr = f.getPath().getName();
832         HColumnDescriptor hcd = htd.getFamily(Bytes.toBytes(familyStr));
833         // verify that the compression on this file matches the configured
834         // compression
835         Path dataFilePath = fs.listStatus(f.getPath())[0].getPath();
836         Reader reader = HFile.createReader(fs, dataFilePath, new CacheConfig(conf));
837         Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
838 
839         byte[] bloomFilter = fileInfo.get(StoreFile.BLOOM_FILTER_TYPE_KEY);
840         if (bloomFilter == null) bloomFilter = Bytes.toBytes("NONE");
841         assertEquals("Incorrect bloom filter used for column family " + familyStr +
842           "(reader: " + reader + ")",
843           hcd.getBloomFilterType(), BloomType.valueOf(Bytes.toString(bloomFilter)));
844         assertEquals("Incorrect compression used for column family " + familyStr +
845           "(reader: " + reader + ")", hcd.getCompression(), reader.getCompressionAlgorithm());
846       }
847     } finally {
848       dir.getFileSystem(conf).delete(dir, true);
849     }
850   }
851 
852   /**
853    * Write random values to the writer assuming a table created using
854    * {@link #FAMILIES} as column family descriptors
855    */
856   private void writeRandomKeyValues(RecordWriter<ImmutableBytesWritable, KeyValue> writer,
857       TaskAttemptContext context, Set<byte[]> families, int numRows)
858       throws IOException, InterruptedException {
859     byte keyBytes[] = new byte[Bytes.SIZEOF_INT];
860     int valLength = 10;
861     byte valBytes[] = new byte[valLength];
862 
863     int taskId = context.getTaskAttemptID().getTaskID().getId();
864     assert taskId < Byte.MAX_VALUE : "Unit tests dont support > 127 tasks!";
865 
866     Random random = new Random();
867     for (int i = 0; i < numRows; i++) {
868 
869       Bytes.putInt(keyBytes, 0, i);
870       random.nextBytes(valBytes);
871       ImmutableBytesWritable key = new ImmutableBytesWritable(keyBytes);
872 
873       for (byte[] family : families) {
874         KeyValue kv = new KeyValue(keyBytes, family,
875             PerformanceEvaluation.QUALIFIER_NAME, valBytes);
876         writer.write(key, kv);
877       }
878     }
879   }
880 
881   /**
882    * This test is to test the scenario happened in HBASE-6901.
883    * All files are bulk loaded and excluded from minor compaction.
884    * Without the fix of HBASE-6901, an ArrayIndexOutOfBoundsException
885    * will be thrown.
886    */
887   @Test
888   public void testExcludeAllFromMinorCompaction() throws Exception {
889     Configuration conf = util.getConfiguration();
890     conf.setInt("hbase.hstore.compaction.min", 2);
891     generateRandomStartKeys(5);
892 
893     try {
894       util.startMiniCluster();
895       final FileSystem fs = util.getDFSCluster().getFileSystem();
896       HBaseAdmin admin = new HBaseAdmin(conf);
897       HTable table = util.createTable(TABLE_NAME, FAMILIES);
898       assertEquals("Should start with empty table", 0, util.countRows(table));
899 
900       // deep inspection: get the StoreFile dir
901       final Path storePath = Store.getStoreHomedir(
902           HTableDescriptor.getTableDir(FSUtils.getRootDir(conf), TABLE_NAME),
903           admin.getTableRegions(TABLE_NAME).get(0).getEncodedName(),
904           FAMILIES[0]);
905       assertEquals(0, fs.listStatus(storePath).length);
906 
907       // Generate two bulk load files
908       conf.setBoolean("hbase.mapreduce.hfileoutputformat.compaction.exclude",
909           true);
910       util.startMiniMapReduceCluster();
911 
912       for (int i = 0; i < 2; i++) {
913         Path testDir = util.getDataTestDir("testExcludeAllFromMinorCompaction_" + i);
914         runIncrementalPELoad(conf, table, testDir);
915         // Perform the actual load
916         new LoadIncrementalHFiles(conf).doBulkLoad(testDir, table);
917       }
918 
919       // Ensure data shows up
920       int expectedRows = 2 * NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;
921       assertEquals("LoadIncrementalHFiles should put expected data in table",
922           expectedRows, util.countRows(table));
923 
924       // should have a second StoreFile now
925       assertEquals(2, fs.listStatus(storePath).length);
926 
927       // minor compactions shouldn't get rid of the file
928       admin.compact(TABLE_NAME);
929       try {
930         quickPoll(new Callable<Boolean>() {
931           public Boolean call() throws Exception {
932             return fs.listStatus(storePath).length == 1;
933           }
934         }, 5000);
935         throw new IOException("SF# = " + fs.listStatus(storePath).length);
936       } catch (AssertionError ae) {
937         // this is expected behavior
938       }
939 
940       // a major compaction should work though
941       admin.majorCompact(TABLE_NAME);
942       quickPoll(new Callable<Boolean>() {
943         public Boolean call() throws Exception {
944           return fs.listStatus(storePath).length == 1;
945         }
946       }, 5000);
947 
948     } finally {
949       util.shutdownMiniMapReduceCluster();
950       util.shutdownMiniCluster();
951     }
952   }
953 
954   @Test
955   public void testExcludeMinorCompaction() throws Exception {
956     Configuration conf = util.getConfiguration();
957     conf.setInt("hbase.hstore.compaction.min", 2);
958     Path testDir = util.getDataTestDir("testExcludeMinorCompaction");
959     generateRandomStartKeys(5);
960 
961     try {
962       util.startMiniCluster();
963       final FileSystem fs = util.getDFSCluster().getFileSystem();
964       HBaseAdmin admin = new HBaseAdmin(conf);
965       HTable table = util.createTable(TABLE_NAME, FAMILIES);
966       assertEquals("Should start with empty table", 0, util.countRows(table));
967 
968       // deep inspection: get the StoreFile dir
969       final Path storePath = Store.getStoreHomedir(
970           HTableDescriptor.getTableDir(FSUtils.getRootDir(conf), TABLE_NAME),
971           admin.getTableRegions(TABLE_NAME).get(0).getEncodedName(),
972           FAMILIES[0]);
973       assertEquals(0, fs.listStatus(storePath).length);
974 
975       // put some data in it and flush to create a storefile
976       Put p = new Put(Bytes.toBytes("test"));
977       p.add(FAMILIES[0], Bytes.toBytes("1"), Bytes.toBytes("1"));
978       table.put(p);
979       admin.flush(TABLE_NAME);
980       assertEquals(1, util.countRows(table));
981       quickPoll(new Callable<Boolean>() {
982         public Boolean call() throws Exception {
983           return fs.listStatus(storePath).length == 1;
984         }
985       }, 5000);
986 
987       // Generate a bulk load file with more rows
988       conf.setBoolean("hbase.mapreduce.hfileoutputformat.compaction.exclude",
989           true);
990       util.startMiniMapReduceCluster();
991       runIncrementalPELoad(conf, table, testDir);
992 
993       // Perform the actual load
994       new LoadIncrementalHFiles(conf).doBulkLoad(testDir, table);
995 
996       // Ensure data shows up
997       int expectedRows = NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;
998       assertEquals("LoadIncrementalHFiles should put expected data in table",
999           expectedRows + 1, util.countRows(table));
1000 
1001       // should have a second StoreFile now
1002       assertEquals(2, fs.listStatus(storePath).length);
1003 
1004       // minor compactions shouldn't get rid of the file
1005       admin.compact(TABLE_NAME);
1006       try {
1007         quickPoll(new Callable<Boolean>() {
1008           public Boolean call() throws Exception {
1009             return fs.listStatus(storePath).length == 1;
1010           }
1011         }, 5000);
1012         throw new IOException("SF# = " + fs.listStatus(storePath).length);
1013       } catch (AssertionError ae) {
1014         // this is expected behavior
1015       }
1016 
1017       // a major compaction should work though
1018       admin.majorCompact(TABLE_NAME);
1019       quickPoll(new Callable<Boolean>() {
1020         public Boolean call() throws Exception {
1021           return fs.listStatus(storePath).length == 1;
1022         }
1023       }, 5000);
1024 
1025     } finally {
1026       util.shutdownMiniMapReduceCluster();
1027       util.shutdownMiniCluster();
1028     }
1029   }
1030 
1031   private void quickPoll(Callable<Boolean> c, int waitMs) throws Exception {
1032     int sleepMs = 10;
1033     int retries = (int) Math.ceil(((double) waitMs) / sleepMs);
1034     while (retries-- > 0) {
1035       if (c.call().booleanValue()) {
1036         return;
1037       }
1038       Thread.sleep(sleepMs);
1039     }
1040     fail();
1041   }
1042 
1043   public static void main(String args[]) throws Exception {
1044     new TestHFileOutputFormat().manualTest(args);
1045   }
1046 
1047   public void manualTest(String args[]) throws Exception {
1048     Configuration conf = HBaseConfiguration.create();
1049     util = new HBaseTestingUtility(conf);
1050     if ("newtable".equals(args[0])) {
1051       byte[] tname = args[1].getBytes();
1052       HTable table = util.createTable(tname, FAMILIES);
1053       HBaseAdmin admin = new HBaseAdmin(conf);
1054       admin.disableTable(tname);
1055       byte[][] startKeys = generateRandomStartKeys(5);
1056       util.createMultiRegions(conf, table, FAMILIES[0], startKeys);
1057       admin.enableTable(tname);
1058     } else if ("incremental".equals(args[0])) {
1059       byte[] tname = args[1].getBytes();
1060       HTable table = new HTable(conf, tname);
1061       Path outDir = new Path("incremental-out");
1062       runIncrementalPELoad(conf, table, outDir);
1063     } else {
1064       throw new RuntimeException(
1065           "usage: TestHFileOutputFormat newtable | incremental");
1066     }
1067   }
1068 
1069   @org.junit.Rule
1070   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
1071     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
1072 }
1073