1   /**
2    * Copyright 2007 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.regionserver;
21  
22  import java.io.IOException;
23  import java.nio.ByteBuffer;
24  import java.util.ArrayList;
25  import java.util.Arrays;
26  import java.util.Collections;
27  import java.util.Comparator;
28  import java.util.List;
29  import java.util.TreeSet;
30  
31  import org.apache.commons.logging.Log;
32  import org.apache.commons.logging.LogFactory;
33  import org.apache.hadoop.conf.Configuration;
34  import org.apache.hadoop.fs.FileSystem;
35  import org.apache.hadoop.fs.Path;
36  import org.apache.hadoop.hbase.HBaseTestCase;
37  import org.apache.hadoop.hbase.HBaseTestingUtility;
38  import org.apache.hadoop.hbase.HConstants;
39  import org.apache.hadoop.hbase.KeyValue;
40  import org.apache.hadoop.hbase.client.Scan;
41  import org.apache.hadoop.hbase.io.Reference.Range;
42  import org.apache.hadoop.hbase.io.hfile.BlockCache;
43  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
44  import org.apache.hadoop.hbase.io.hfile.CacheStats;
45  import org.apache.hadoop.hbase.io.hfile.HFile;
46  import org.apache.hadoop.hbase.io.hfile.HFileScanner;
47  import org.apache.hadoop.hbase.util.BloomFilterFactory;
48  import org.apache.hadoop.hbase.util.Bytes;
49  import org.apache.hadoop.hdfs.MiniDFSCluster;
50  import org.mockito.Mockito;
51  
52  import com.google.common.base.Joiner;
53  import com.google.common.collect.Iterables;
54  import com.google.common.collect.Lists;
55  
56  /**
57   * Test HStoreFile
58   */
59  public class TestStoreFile extends HBaseTestCase {
60    static final Log LOG = LogFactory.getLog(TestStoreFile.class);
61    private CacheConfig cacheConf =  new CacheConfig(conf);
62    private String ROOT_DIR;
63  
64    @Override
65    public void setUp() throws Exception {
66      super.setUp();
67      ROOT_DIR = new Path(this.testDir, "TestStoreFile").toString();
68    }
69  
70    /**
71     * Write a file and then assert that we can read from top and bottom halves
72     * using two HalfMapFiles.
73     * @throws Exception
74     */
75    public void testBasicHalfMapFile() throws Exception {
76      // Make up a directory hierarchy that has a regiondir and familyname.
77      StoreFile.Writer writer = StoreFile.createWriter(this.fs,
78        new Path(new Path(this.testDir, "regionname"), "familyname"), 2 * 1024,
79        conf, cacheConf);
80      writeStoreFile(writer);
81      checkHalfHFile(new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
82          StoreFile.BloomType.NONE));
83    }
84  
85    private void writeStoreFile(final StoreFile.Writer writer) throws IOException {
86      writeStoreFile(writer, Bytes.toBytes(getName()), Bytes.toBytes(getName()));
87    }
88    /*
89     * Writes HStoreKey and ImmutableBytes data to passed writer and
90     * then closes it.
91     * @param writer
92     * @throws IOException
93     */
94    public static void writeStoreFile(final StoreFile.Writer writer, byte[] fam, byte[] qualifier)
95    throws IOException {
96      long now = System.currentTimeMillis();
97      try {
98        for (char d = FIRST_CHAR; d <= LAST_CHAR; d++) {
99          for (char e = FIRST_CHAR; e <= LAST_CHAR; e++) {
100           byte[] b = new byte[] { (byte) d, (byte) e };
101           writer.append(new KeyValue(b, fam, qualifier, now, b));
102         }
103       }
104     } finally {
105       writer.close();
106     }
107   }
108 
109   /**
110    * Test that our mechanism of writing store files in one region to reference
111    * store files in other regions works.
112    * @throws IOException
113    */
114   public void testReference()
115   throws IOException {
116     Path storedir = new Path(new Path(this.testDir, "regionname"), "familyname");
117     Path dir = new Path(storedir, "1234567890");
118     // Make a store file and write data to it.
119     StoreFile.Writer writer = StoreFile.createWriter(this.fs, dir, 8 * 1024,
120         conf, cacheConf);
121     writeStoreFile(writer);
122     StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
123         StoreFile.BloomType.NONE);
124     StoreFile.Reader reader = hsf.createReader();
125     // Split on a row, not in middle of row.  Midkey returned by reader
126     // may be in middle of row.  Create new one with empty column and
127     // timestamp.
128     KeyValue kv = KeyValue.createKeyValueFromKey(reader.midkey());
129     byte [] midRow = kv.getRow();
130     kv = KeyValue.createKeyValueFromKey(reader.getLastKey());
131     byte [] finalRow = kv.getRow();
132     // Make a reference
133     Path refPath = StoreFile.split(fs, dir, hsf, midRow, Range.top);
134     StoreFile refHsf = new StoreFile(this.fs, refPath, conf, cacheConf,
135         StoreFile.BloomType.NONE);
136     // Now confirm that I can read from the reference and that it only gets
137     // keys from top half of the file.
138     HFileScanner s = refHsf.createReader().getScanner(false, false);
139     for(boolean first = true; (!s.isSeeked() && s.seekTo()) || s.next();) {
140       ByteBuffer bb = s.getKey();
141       kv = KeyValue.createKeyValueFromKey(bb);
142       if (first) {
143         assertTrue(Bytes.equals(kv.getRow(), midRow));
144         first = false;
145       }
146     }
147     assertTrue(Bytes.equals(kv.getRow(), finalRow));
148   }
149 
150   private void checkHalfHFile(final StoreFile f)
151   throws IOException {
152     byte [] midkey = f.createReader().midkey();
153     KeyValue midKV = KeyValue.createKeyValueFromKey(midkey);
154     byte [] midRow = midKV.getRow();
155     // Create top split.
156     Path topDir = Store.getStoreHomedir(this.testDir, "1",
157       Bytes.toBytes(f.getPath().getParent().getName()));
158     if (this.fs.exists(topDir)) {
159       this.fs.delete(topDir, true);
160     }
161     Path topPath = StoreFile.split(this.fs, topDir, f, midRow, Range.top);
162     // Create bottom split.
163     Path bottomDir = Store.getStoreHomedir(this.testDir, "2",
164       Bytes.toBytes(f.getPath().getParent().getName()));
165     if (this.fs.exists(bottomDir)) {
166       this.fs.delete(bottomDir, true);
167     }
168     Path bottomPath = StoreFile.split(this.fs, bottomDir,
169       f, midRow, Range.bottom);
170     // Make readers on top and bottom.
171     StoreFile.Reader top = new StoreFile(this.fs, topPath, conf, cacheConf,
172         StoreFile.BloomType.NONE).createReader();
173     StoreFile.Reader bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf,
174         StoreFile.BloomType.NONE).createReader();
175     ByteBuffer previous = null;
176     LOG.info("Midkey: " + midKV.toString());
177     ByteBuffer bbMidkeyBytes = ByteBuffer.wrap(midkey);
178     try {
179       // Now make two HalfMapFiles and assert they can read the full backing
180       // file, one from the top and the other from the bottom.
181       // Test bottom half first.
182       // Now test reading from the top.
183       boolean first = true;
184       ByteBuffer key = null;
185       HFileScanner topScanner = top.getScanner(false, false);
186       while ((!topScanner.isSeeked() && topScanner.seekTo()) ||
187           (topScanner.isSeeked() && topScanner.next())) {
188         key = topScanner.getKey();
189 
190         if (topScanner.getReader().getComparator().compare(key.array(),
191           key.arrayOffset(), key.limit(), midkey, 0, midkey.length) < 0) {
192           fail("key=" + Bytes.toStringBinary(key) + " < midkey=" +
193               Bytes.toStringBinary(midkey));
194         }
195         if (first) {
196           first = false;
197           LOG.info("First in top: " + Bytes.toString(Bytes.toBytes(key)));
198         }
199       }
200       LOG.info("Last in top: " + Bytes.toString(Bytes.toBytes(key)));
201 
202       first = true;
203       HFileScanner bottomScanner = bottom.getScanner(false, false);
204       while ((!bottomScanner.isSeeked() && bottomScanner.seekTo()) ||
205           bottomScanner.next()) {
206         previous = bottomScanner.getKey();
207         key = bottomScanner.getKey();
208         if (first) {
209           first = false;
210           LOG.info("First in bottom: " +
211             Bytes.toString(Bytes.toBytes(previous)));
212         }
213         assertTrue(key.compareTo(bbMidkeyBytes) < 0);
214       }
215       if (previous != null) {
216         LOG.info("Last in bottom: " + Bytes.toString(Bytes.toBytes(previous)));
217       }
218       // Remove references.
219       this.fs.delete(topPath, false);
220       this.fs.delete(bottomPath, false);
221 
222       // Next test using a midkey that does not exist in the file.
223       // First, do a key that is < than first key. Ensure splits behave
224       // properly.
225       byte [] badmidkey = Bytes.toBytes("  .");
226       topPath = StoreFile.split(this.fs, topDir, f, badmidkey, Range.top);
227       bottomPath = StoreFile.split(this.fs, bottomDir, f, badmidkey,
228         Range.bottom);
229       top = new StoreFile(this.fs, topPath, conf, cacheConf,
230           StoreFile.BloomType.NONE).createReader();
231       bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf,
232           StoreFile.BloomType.NONE).createReader();
233       bottomScanner = bottom.getScanner(false, false);
234       int count = 0;
235       while ((!bottomScanner.isSeeked() && bottomScanner.seekTo()) ||
236           bottomScanner.next()) {
237         count++;
238       }
239       // When badkey is < than the bottom, should return no values.
240       assertTrue(count == 0);
241       // Now read from the top.
242       first = true;
243       topScanner = top.getScanner(false, false);
244       while ((!topScanner.isSeeked() && topScanner.seekTo()) ||
245           topScanner.next()) {
246         key = topScanner.getKey();
247         assertTrue(topScanner.getReader().getComparator().compare(key.array(),
248           key.arrayOffset(), key.limit(), badmidkey, 0, badmidkey.length) >= 0);
249         if (first) {
250           first = false;
251           KeyValue keyKV = KeyValue.createKeyValueFromKey(key);
252           LOG.info("First top when key < bottom: " + keyKV);
253           String tmp = Bytes.toString(keyKV.getRow());
254           for (int i = 0; i < tmp.length(); i++) {
255             assertTrue(tmp.charAt(i) == 'a');
256           }
257         }
258       }
259       KeyValue keyKV = KeyValue.createKeyValueFromKey(key);
260       LOG.info("Last top when key < bottom: " + keyKV);
261       String tmp = Bytes.toString(keyKV.getRow());
262       for (int i = 0; i < tmp.length(); i++) {
263         assertTrue(tmp.charAt(i) == 'z');
264       }
265       // Remove references.
266       this.fs.delete(topPath, false);
267       this.fs.delete(bottomPath, false);
268 
269       // Test when badkey is > than last key in file ('||' > 'zz').
270       badmidkey = Bytes.toBytes("|||");
271       topPath = StoreFile.split(this.fs, topDir, f, badmidkey, Range.top);
272       bottomPath = StoreFile.split(this.fs, bottomDir, f, badmidkey,
273         Range.bottom);
274       top = new StoreFile(this.fs, topPath, conf, cacheConf,
275           StoreFile.BloomType.NONE).createReader();
276       bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf,
277           StoreFile.BloomType.NONE).createReader();
278       first = true;
279       bottomScanner = bottom.getScanner(false, false);
280       while ((!bottomScanner.isSeeked() && bottomScanner.seekTo()) ||
281           bottomScanner.next()) {
282         key = bottomScanner.getKey();
283         if (first) {
284           first = false;
285           keyKV = KeyValue.createKeyValueFromKey(key);
286           LOG.info("First bottom when key > top: " + keyKV);
287           tmp = Bytes.toString(keyKV.getRow());
288           for (int i = 0; i < tmp.length(); i++) {
289             assertTrue(tmp.charAt(i) == 'a');
290           }
291         }
292       }
293       keyKV = KeyValue.createKeyValueFromKey(key);
294       LOG.info("Last bottom when key > top: " + keyKV);
295       for (int i = 0; i < tmp.length(); i++) {
296         assertTrue(Bytes.toString(keyKV.getRow()).charAt(i) == 'z');
297       }
298       count = 0;
299       topScanner = top.getScanner(false, false);
300       while ((!topScanner.isSeeked() && topScanner.seekTo()) ||
301           (topScanner.isSeeked() && topScanner.next())) {
302         count++;
303       }
304       // When badkey is < than the bottom, should return no values.
305       assertTrue(count == 0);
306     } finally {
307       if (top != null) {
308         top.close(true); // evict since we are about to delete the file
309       }
310       if (bottom != null) {
311         bottom.close(true); // evict since we are about to delete the file
312       }
313       fs.delete(f.getPath(), true);
314     }
315   }
316 
317   private static final String localFormatter = "%010d";
318 
319   private void bloomWriteRead(StoreFile.Writer writer, FileSystem fs)
320   throws Exception {
321     float err = conf.getFloat(
322         BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, 0);
323     Path f = writer.getPath();
324     long now = System.currentTimeMillis();
325     for (int i = 0; i < 2000; i += 2) {
326       String row = String.format(localFormatter, i);
327       KeyValue kv = new KeyValue(row.getBytes(), "family".getBytes(),
328         "col".getBytes(), now, "value".getBytes());
329       writer.append(kv);
330     }
331     writer.close();
332 
333     StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf);
334     reader.loadFileInfo();
335     reader.loadBloomfilter();
336     StoreFileScanner scanner = reader.getStoreFileScanner(false, false);
337 
338     // check false positives rate
339     int falsePos = 0;
340     int falseNeg = 0;
341     for (int i = 0; i < 2000; i++) {
342       String row = String.format(localFormatter, i);
343       TreeSet<byte[]> columns = new TreeSet<byte[]>();
344       columns.add("family:col".getBytes());
345 
346       Scan scan = new Scan(row.getBytes(),row.getBytes());
347       scan.addColumn("family".getBytes(), "family:col".getBytes());
348       boolean exists = scanner.shouldSeek(scan, columns);
349       if (i % 2 == 0) {
350         if (!exists) falseNeg++;
351       } else {
352         if (exists) falsePos++;
353       }
354     }
355     reader.close(true); // evict because we are about to delete the file
356     fs.delete(f, true);
357     assertEquals("False negatives: " + falseNeg, 0, falseNeg);
358     int maxFalsePos = (int) (2 * 2000 * err);
359     assertTrue("Too many false positives: " + falsePos + " (err=" + err
360         + ", expected no more than " + maxFalsePos + ")",
361         falsePos <= maxFalsePos);
362   }
363 
364   public void testBloomFilter() throws Exception {
365     FileSystem fs = FileSystem.getLocal(conf);
366     conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE,
367         (float) 0.01);
368     conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
369 
370     // write the file
371     Path f = new Path(ROOT_DIR, getName());
372     StoreFile.Writer writer = new StoreFile.Writer(fs, f,
373         StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM,
374         conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000);
375 
376     bloomWriteRead(writer, fs);
377   }
378 
379   public void testBloomTypes() throws Exception {
380     float err = (float) 0.01;
381     FileSystem fs = FileSystem.getLocal(conf);
382     conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, err);
383     conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
384 
385     int rowCount = 50;
386     int colCount = 10;
387     int versions = 2;
388 
389     // run once using columns and once using rows
390     StoreFile.BloomType[] bt =
391       {StoreFile.BloomType.ROWCOL, StoreFile.BloomType.ROW};
392     int[] expKeys    = {rowCount*colCount, rowCount};
393     // below line deserves commentary.  it is expected bloom false positives
394     //  column = rowCount*2*colCount inserts
395     //  row-level = only rowCount*2 inserts, but failures will be magnified by
396     //              2nd for loop for every column (2*colCount)
397     float[] expErr   = {2*rowCount*colCount*err, 2*rowCount*2*colCount*err};
398 
399     for (int x : new int[]{0,1}) {
400       // write the file
401       Path f = new Path(ROOT_DIR, getName() + x);
402       StoreFile.Writer writer = new StoreFile.Writer(fs, f,
403           StoreFile.DEFAULT_BLOCKSIZE_SMALL,
404           HFile.DEFAULT_COMPRESSION_ALGORITHM,
405           conf, cacheConf, KeyValue.COMPARATOR, bt[x], expKeys[x]);
406 
407       long now = System.currentTimeMillis();
408       for (int i = 0; i < rowCount*2; i += 2) { // rows
409         for (int j = 0; j < colCount*2; j += 2) {   // column qualifiers
410           String row = String.format(localFormatter, i);
411           String col = String.format(localFormatter, j);
412           for (int k= 0; k < versions; ++k) { // versions
413             KeyValue kv = new KeyValue(row.getBytes(),
414               "family".getBytes(), ("col" + col).getBytes(),
415                 now-k, Bytes.toBytes((long)-1));
416             writer.append(kv);
417           }
418         }
419       }
420       writer.close();
421 
422       StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf);
423       reader.loadFileInfo();
424       reader.loadBloomfilter();
425       StoreFileScanner scanner = reader.getStoreFileScanner(true, true);
426       assertEquals(expKeys[x], reader.bloomFilter.getKeyCount());
427 
428       // check false positives rate
429       int falsePos = 0;
430       int falseNeg = 0;
431       for (int i = 0; i < rowCount*2; ++i) { // rows
432         for (int j = 0; j < colCount*2; ++j) {   // column qualifiers
433           String row = String.format(localFormatter, i);
434           String col = String.format(localFormatter, j);
435           TreeSet<byte[]> columns = new TreeSet<byte[]>();
436           columns.add(("col" + col).getBytes());
437 
438           Scan scan = new Scan(row.getBytes(),row.getBytes());
439           scan.addColumn("family".getBytes(), ("col"+col).getBytes());
440           boolean exists = scanner.shouldSeek(scan, columns);
441           boolean shouldRowExist = i % 2 == 0;
442           boolean shouldColExist = j % 2 == 0;
443           shouldColExist = shouldColExist || bt[x] == StoreFile.BloomType.ROW;
444           if (shouldRowExist && shouldColExist) {
445             if (!exists) falseNeg++;
446           } else {
447             if (exists) falsePos++;
448           }
449         }
450       }
451       reader.close(true); // evict because we are about to delete the file
452       fs.delete(f, true);
453       System.out.println(bt[x].toString());
454       System.out.println("  False negatives: " + falseNeg);
455       System.out.println("  False positives: " + falsePos);
456       assertEquals(0, falseNeg);
457       assertTrue(falsePos < 2*expErr[x]);
458     }
459   }
460 
461   public void testBloomEdgeCases() throws Exception {
462     float err = (float)0.005;
463     FileSystem fs = FileSystem.getLocal(conf);
464     Path f = new Path(ROOT_DIR, getName());
465     conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, err);
466     conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
467     conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_MAX_KEYS, 1000);
468 
469     // This test only runs for HFile format version 1.
470     conf.setInt(HFile.FORMAT_VERSION_KEY, 1);
471 
472     // this should not create a bloom because the max keys is too small
473     StoreFile.Writer writer = new StoreFile.Writer(fs, f,
474         StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM,
475         conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000);
476     assertFalse(writer.hasBloom());
477     writer.close();
478     fs.delete(f, true);
479 
480     conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_MAX_KEYS,
481         Integer.MAX_VALUE);
482 
483     // TODO: commented out because we run out of java heap space on trunk
484     /*
485     // the below config caused IllegalArgumentException in our production cluster
486     // however, the resulting byteSize is < MAX_INT, so this should work properly
487     writer = new StoreFile.Writer(fs, f,
488         StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM,
489         conf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 272446963);
490     assertTrue(writer.hasBloom());
491     bloomWriteRead(writer, fs);
492     */
493 
494     // this, however, is too large and should not create a bloom
495     // because Java can't create a contiguous array > MAX_INT
496     writer = new StoreFile.Writer(fs, f,
497         StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM,
498         conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW,
499         Integer.MAX_VALUE);
500     assertFalse(writer.hasBloom());
501     writer.close();
502     fs.delete(f, true);
503   }
504 
505   public void testFlushTimeComparator() {
506     assertOrdering(StoreFile.Comparators.FLUSH_TIME,
507         mockStoreFile(true, 1000, -1, "/foo/123"),
508         mockStoreFile(true, 1000, -1, "/foo/126"),
509         mockStoreFile(true, 2000, -1, "/foo/126"),
510         mockStoreFile(false, -1, 1, "/foo/1"),
511         mockStoreFile(false, -1, 3, "/foo/2"),
512         mockStoreFile(false, -1, 5, "/foo/2"),
513         mockStoreFile(false, -1, 5, "/foo/3"));
514   }
515 
516   /**
517    * Assert that the given comparator orders the given storefiles in the
518    * same way that they're passed.
519    */
520   private void assertOrdering(Comparator<StoreFile> comparator, StoreFile ... sfs) {
521     ArrayList<StoreFile> sorted = Lists.newArrayList(sfs);
522     Collections.shuffle(sorted);
523     Collections.sort(sorted, comparator);
524     LOG.debug("sfs: " + Joiner.on(",").join(sfs));
525     LOG.debug("sorted: " + Joiner.on(",").join(sorted));
526     assertTrue(Iterables.elementsEqual(Arrays.asList(sfs), sorted));
527   }
528 
529   /**
530    * Create a mock StoreFile with the given attributes.
531    */
532   private StoreFile mockStoreFile(boolean bulkLoad, long bulkTimestamp,
533       long seqId, String path) {
534     StoreFile mock = Mockito.mock(StoreFile.class);
535     Mockito.doReturn(bulkLoad).when(mock).isBulkLoadResult();
536     Mockito.doReturn(bulkTimestamp).when(mock).getBulkLoadTimestamp();
537     if (bulkLoad) {
538       // Bulk load files will throw if you ask for their sequence ID
539       Mockito.doThrow(new IllegalAccessError("bulk load"))
540         .when(mock).getMaxSequenceId();
541     } else {
542       Mockito.doReturn(seqId).when(mock).getMaxSequenceId();
543     }
544     Mockito.doReturn(new Path(path)).when(mock).getPath();
545     String name = "mock storefile, bulkLoad=" + bulkLoad +
546       " bulkTimestamp=" + bulkTimestamp +
547       " seqId=" + seqId +
548       " path=" + path;
549     Mockito.doReturn(name).when(mock).toString();
550     return mock;
551   }
552 
553   /**
554    * Generate a list of KeyValues for testing based on given parameters
555    * @param timestamps
556    * @param numRows
557    * @param qualifier
558    * @param family
559    * @return
560    */
561   List<KeyValue> getKeyValueSet(long[] timestamps, int numRows,
562       byte[] qualifier, byte[] family) {
563     List<KeyValue> kvList = new ArrayList<KeyValue>();
564     for (int i=1;i<=numRows;i++) {
565       byte[] b = Bytes.toBytes(i) ;
566       LOG.info(Bytes.toString(b));
567       LOG.info(Bytes.toString(b));
568       for (long timestamp: timestamps)
569       {
570         kvList.add(new KeyValue(b, family, qualifier, timestamp, b));
571       }
572     }
573     return kvList;
574   }
575 
576   /**
577    * Test to ensure correctness when using StoreFile with multiple timestamps
578    * @throws IOException
579    */
580   public void testMultipleTimestamps() throws IOException {
581     byte[] family = Bytes.toBytes("familyname");
582     byte[] qualifier = Bytes.toBytes("qualifier");
583     int numRows = 10;
584     long[] timestamps = new long[] {20,10,5,1};
585     Scan scan = new Scan();
586 
587     Path storedir = new Path(new Path(this.testDir, "regionname"),
588     "familyname");
589     Path dir = new Path(storedir, "1234567890");
590     StoreFile.Writer writer = StoreFile.createWriter(this.fs, dir, 8 * 1024,
591         conf, cacheConf);
592 
593     List<KeyValue> kvList = getKeyValueSet(timestamps,numRows,
594         family, qualifier);
595 
596     for (KeyValue kv : kvList) {
597       writer.append(kv);
598     }
599     writer.appendMetadata(0, false);
600     writer.close();
601 
602     StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
603         StoreFile.BloomType.NONE);
604     StoreFile.Reader reader = hsf.createReader();
605     StoreFileScanner scanner = reader.getStoreFileScanner(false, false);
606     TreeSet<byte[]> columns = new TreeSet<byte[]>();
607     columns.add(qualifier);
608 
609     scan.setTimeRange(20, 100);
610     assertTrue(scanner.shouldSeek(scan, columns));
611 
612     scan.setTimeRange(1, 2);
613     assertTrue(scanner.shouldSeek(scan, columns));
614 
615     scan.setTimeRange(8, 10);
616     assertTrue(scanner.shouldSeek(scan, columns));
617 
618     scan.setTimeRange(7, 50);
619     assertTrue(scanner.shouldSeek(scan, columns));
620 
621     /*This test is not required for correctness but it should pass when
622      * timestamp range optimization is on*/
623     //scan.setTimeRange(27, 50);
624     //assertTrue(!scanner.shouldSeek(scan, columns));
625   }
626 
627   public void testCacheOnWriteEvictOnClose() throws Exception {
628     Configuration conf = this.conf;
629 
630     // Find a home for our files
631     Path baseDir = new Path(new Path(this.testDir, "regionname"),"twoCOWEOC");
632 
633     // Grab the block cache and get the initial hit/miss counts
634     BlockCache bc = new CacheConfig(conf).getBlockCache();
635     assertNotNull(bc);
636     CacheStats cs = bc.getStats();
637     long startHit = cs.getHitCount();
638     long startMiss = cs.getMissCount();
639     long startEvicted = cs.getEvictedCount();
640 
641     // Let's write a StoreFile with three blocks, with cache on write off
642     conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, false);
643     CacheConfig cacheConf = new CacheConfig(conf);
644     Path pathCowOff = new Path(baseDir, "123456789");
645     StoreFile.Writer writer = writeStoreFile(conf, cacheConf, pathCowOff, 3);
646     StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
647         StoreFile.BloomType.NONE);
648     LOG.debug(hsf.getPath().toString());
649 
650     // Read this file, we should see 3 misses
651     StoreFile.Reader reader = hsf.createReader();
652     reader.loadFileInfo();
653     StoreFileScanner scanner = reader.getStoreFileScanner(true, true);
654     scanner.seek(KeyValue.LOWESTKEY);
655     while (scanner.next() != null);
656     assertEquals(startHit, cs.getHitCount());
657     assertEquals(startMiss + 3, cs.getMissCount());
658     assertEquals(startEvicted, cs.getEvictedCount());
659     startMiss += 3;
660     scanner.close();
661     reader.close(cacheConf.shouldEvictOnClose());
662 
663     // Now write a StoreFile with three blocks, with cache on write on
664     conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, true);
665     cacheConf = new CacheConfig(conf);
666     Path pathCowOn = new Path(baseDir, "123456788");
667     writer = writeStoreFile(conf, cacheConf, pathCowOn, 3);
668     hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
669         StoreFile.BloomType.NONE);
670 
671     // Read this file, we should see 3 hits
672     reader = hsf.createReader();
673     scanner = reader.getStoreFileScanner(true, true);
674     scanner.seek(KeyValue.LOWESTKEY);
675     while (scanner.next() != null);
676     assertEquals(startHit + 3, cs.getHitCount());
677     assertEquals(startMiss, cs.getMissCount());
678     assertEquals(startEvicted, cs.getEvictedCount());
679     startHit += 3;
680     scanner.close();
681     reader.close(cacheConf.shouldEvictOnClose());
682 
683     // Let's read back the two files to ensure the blocks exactly match
684     hsf = new StoreFile(this.fs, pathCowOff, conf, cacheConf,
685         StoreFile.BloomType.NONE);
686     StoreFile.Reader readerOne = hsf.createReader();
687     readerOne.loadFileInfo();
688     StoreFileScanner scannerOne = readerOne.getStoreFileScanner(true, true);
689     scannerOne.seek(KeyValue.LOWESTKEY);
690     hsf = new StoreFile(this.fs, pathCowOn, conf, cacheConf,
691         StoreFile.BloomType.NONE);
692     StoreFile.Reader readerTwo = hsf.createReader();
693     readerTwo.loadFileInfo();
694     StoreFileScanner scannerTwo = readerTwo.getStoreFileScanner(true, true);
695     scannerTwo.seek(KeyValue.LOWESTKEY);
696     KeyValue kv1 = null;
697     KeyValue kv2 = null;
698     while ((kv1 = scannerOne.next()) != null) {
699       kv2 = scannerTwo.next();
700       assertTrue(kv1.equals(kv2));
701       assertTrue(Bytes.compareTo(
702           kv1.getBuffer(), kv1.getKeyOffset(), kv1.getKeyLength(), 
703           kv2.getBuffer(), kv2.getKeyOffset(), kv2.getKeyLength()) == 0);
704       assertTrue(Bytes.compareTo(
705           kv1.getBuffer(), kv1.getValueOffset(), kv1.getValueLength(),
706           kv2.getBuffer(), kv2.getValueOffset(), kv2.getValueLength()) == 0);
707     }
708     assertNull(scannerTwo.next());
709     assertEquals(startHit + 6, cs.getHitCount());
710     assertEquals(startMiss, cs.getMissCount());
711     assertEquals(startEvicted, cs.getEvictedCount());
712     startHit += 6;
713     scannerOne.close();
714     readerOne.close(cacheConf.shouldEvictOnClose());
715     scannerTwo.close();
716     readerTwo.close(cacheConf.shouldEvictOnClose());
717 
718     // Let's close the first file with evict on close turned on
719     conf.setBoolean("hbase.rs.evictblocksonclose", true);
720     cacheConf = new CacheConfig(conf);
721     hsf = new StoreFile(this.fs, pathCowOff, conf, cacheConf,
722         StoreFile.BloomType.NONE);
723     reader = hsf.createReader();
724     reader.close(cacheConf.shouldEvictOnClose());
725 
726     // We should have 3 new evictions
727     assertEquals(startHit, cs.getHitCount());
728     assertEquals(startMiss, cs.getMissCount());
729     assertEquals(startEvicted + 3, cs.getEvictedCount());
730     startEvicted += 3;
731 
732     // Let's close the second file with evict on close turned off
733     conf.setBoolean("hbase.rs.evictblocksonclose", false);
734     cacheConf = new CacheConfig(conf);
735     hsf = new StoreFile(this.fs, pathCowOn, conf, cacheConf,
736         StoreFile.BloomType.NONE);
737     reader = hsf.createReader();
738     reader.close(cacheConf.shouldEvictOnClose());
739 
740     // We expect no changes
741     assertEquals(startHit, cs.getHitCount());
742     assertEquals(startMiss, cs.getMissCount());
743     assertEquals(startEvicted, cs.getEvictedCount());
744   }
745 
746   private StoreFile.Writer writeStoreFile(Configuration conf,
747       CacheConfig cacheConf, Path path, int numBlocks)
748   throws IOException {
749     // Let's put ~5 small KVs in each block, so let's make 5*numBlocks KVs
750     int numKVs = 5 * numBlocks;
751     List<KeyValue> kvs = new ArrayList<KeyValue>(numKVs);
752     byte [] b = Bytes.toBytes("x");
753     int totalSize = 0;
754     for (int i=numKVs;i>0;i--) {
755       KeyValue kv = new KeyValue(b, b, b, i, b);
756       kvs.add(kv);
757       // kv has memstoreTS 0, which takes 1 byte to store.
758       totalSize += kv.getLength() + 1;
759     }
760     int blockSize = totalSize / numBlocks;
761     StoreFile.Writer writer = new StoreFile.Writer(fs, path, blockSize,
762         HFile.DEFAULT_COMPRESSION_ALGORITHM,
763         conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.NONE, 2000);
764     // We'll write N-1 KVs to ensure we don't write an extra block
765     kvs.remove(kvs.size()-1);
766     for (KeyValue kv : kvs) {
767       writer.append(kv);
768     }
769     writer.appendMetadata(0, false);
770     writer.close();
771     return writer;
772   }
773 }