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  
21  package org.apache.hadoop.hbase.regionserver;
22  
23  import java.io.IOException;
24  import java.lang.ref.SoftReference;
25  import java.security.PrivilegedExceptionAction;
26  import java.util.ArrayList;
27  import java.util.Collections;
28  import java.util.Iterator;
29  import java.util.List;
30  import java.util.NavigableSet;
31  import java.util.concurrent.ConcurrentSkipListSet;
32  
33  import junit.framework.TestCase;
34  
35  import org.apache.commons.logging.Log;
36  import org.apache.commons.logging.LogFactory;
37  import org.apache.hadoop.conf.Configuration;
38  import org.apache.hadoop.fs.FSDataOutputStream;
39  import org.apache.hadoop.fs.FileStatus;
40  import org.apache.hadoop.fs.FileSystem;
41  import org.apache.hadoop.fs.FileUtil;
42  import org.apache.hadoop.fs.FilterFileSystem;
43  import org.apache.hadoop.fs.LocalFileSystem;
44  import org.apache.hadoop.fs.Path;
45  import org.apache.hadoop.fs.permission.FsPermission;
46  import org.apache.hadoop.hbase.HBaseConfiguration;
47  import org.apache.hadoop.hbase.HBaseTestingUtility;
48  import org.apache.hadoop.hbase.HColumnDescriptor;
49  import org.apache.hadoop.hbase.HConstants;
50  import org.apache.hadoop.hbase.HRegionInfo;
51  import org.apache.hadoop.hbase.HTableDescriptor;
52  import org.apache.hadoop.hbase.KeyValue;
53  import org.apache.hadoop.hbase.MediumTests;
54  import org.apache.hadoop.hbase.client.Get;
55  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
56  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
57  import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
58  import org.apache.hadoop.hbase.regionserver.wal.HLog;
59  import org.apache.hadoop.hbase.security.User;
60  import org.apache.hadoop.hbase.util.Bytes;
61  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
62  import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
63  import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
64  import org.apache.hadoop.util.Progressable;
65  import org.junit.experimental.categories.Category;
66  import org.mockito.Mockito;
67  
68  import com.google.common.base.Joiner;
69  
70  /**
71   * Test class for the Store
72   */
73  @Category(MediumTests.class)
74  public class TestStore extends TestCase {
75    public static final Log LOG = LogFactory.getLog(TestStore.class);
76  
77    Store store;
78    byte [] table = Bytes.toBytes("table");
79    byte [] family = Bytes.toBytes("family");
80  
81    byte [] row = Bytes.toBytes("row");
82    byte [] row2 = Bytes.toBytes("row2");
83    byte [] qf1 = Bytes.toBytes("qf1");
84    byte [] qf2 = Bytes.toBytes("qf2");
85    byte [] qf3 = Bytes.toBytes("qf3");
86    byte [] qf4 = Bytes.toBytes("qf4");
87    byte [] qf5 = Bytes.toBytes("qf5");
88    byte [] qf6 = Bytes.toBytes("qf6");
89  
90    NavigableSet<byte[]> qualifiers =
91      new ConcurrentSkipListSet<byte[]>(Bytes.BYTES_COMPARATOR);
92  
93    List<KeyValue> expected = new ArrayList<KeyValue>();
94    List<KeyValue> result = new ArrayList<KeyValue>();
95  
96    long id = System.currentTimeMillis();
97    Get get = new Get(row);
98  
99    private HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
100   private final String DIR = TEST_UTIL.getDataTestDir("TestStore").toString();
101 
102 
103   /**
104    * Setup
105    * @throws IOException
106    */
107   @Override
108   public void setUp() throws IOException {
109     qualifiers.add(qf1);
110     qualifiers.add(qf3);
111     qualifiers.add(qf5);
112 
113     Iterator<byte[]> iter = qualifiers.iterator();
114     while(iter.hasNext()){
115       byte [] next = iter.next();
116       expected.add(new KeyValue(row, family, next, 1, (byte[])null));
117       get.addColumn(family, next);
118     }
119   }
120 
121   private void init(String methodName) throws IOException {
122     init(methodName, HBaseConfiguration.create());
123   }
124 
125   private void init(String methodName, Configuration conf)
126   throws IOException {
127     HColumnDescriptor hcd = new HColumnDescriptor(family);
128     // some of the tests write 4 versions and then flush
129     // (with HBASE-4241, lower versions are collected on flush)
130     hcd.setMaxVersions(4);
131     init(methodName, conf, hcd);
132   }
133   
134   private void init(String methodName, Configuration conf,
135       HColumnDescriptor hcd) throws IOException {
136     //Setting up a Store
137     Path basedir = new Path(DIR+methodName);
138     Path logdir = new Path(DIR+methodName+"/logs");
139     Path oldLogDir = new Path(basedir, HConstants.HREGION_OLDLOGDIR_NAME);
140     FileSystem fs = FileSystem.get(conf);
141 
142     fs.delete(logdir, true);
143 
144     HTableDescriptor htd = new HTableDescriptor(table);
145     htd.addFamily(hcd);
146     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
147     HLog hlog = new HLog(fs, logdir, oldLogDir, conf);
148     HRegion region = new HRegion(basedir, hlog, fs, conf, info, htd, null);
149 
150     store = new Store(basedir, region, hcd, fs, conf);
151   }
152 
153   public void testDeleteExpiredStoreFiles() throws Exception {
154     int storeFileNum = 4;
155     int ttl = 4;
156     
157     Configuration conf = HBaseConfiguration.create();
158     // Enable the expired store file deletion
159     conf.setBoolean("hbase.store.delete.expired.storefile", true);
160     HColumnDescriptor hcd = new HColumnDescriptor(family);
161     hcd.setTimeToLive(ttl);
162     init(getName(), conf, hcd);
163 
164     long sleepTime = this.store.scanInfo.getTtl() / storeFileNum;
165     long timeStamp;
166     // There are 4 store files and the max time stamp difference among these
167     // store files will be (this.store.ttl / storeFileNum)
168     for (int i = 1; i <= storeFileNum; i++) {
169       LOG.info("Adding some data for the store file #" + i);
170       timeStamp = EnvironmentEdgeManager.currentTimeMillis();
171       this.store.add(new KeyValue(row, family, qf1, timeStamp, (byte[]) null));
172       this.store.add(new KeyValue(row, family, qf2, timeStamp, (byte[]) null));
173       this.store.add(new KeyValue(row, family, qf3, timeStamp, (byte[]) null));
174       flush(i);
175       Thread.sleep(sleepTime);
176     }
177 
178     // Verify the total number of store files
179     assertEquals(storeFileNum, this.store.getStorefiles().size());
180 
181     // Each compaction request will find one expired store file and delete it
182     // by the compaction.
183     for (int i = 1; i <= storeFileNum; i++) {
184       // verify the expired store file.
185       CompactionRequest cr = this.store.requestCompaction();
186       assertEquals(1, cr.getFiles().size());
187       assertTrue(cr.getFiles().get(0).getReader().getMaxTimestamp() < 
188           (System.currentTimeMillis() - this.store.scanInfo.getTtl()));
189       // Verify that the expired the store has been deleted.
190       this.store.compact(cr);
191       assertEquals(storeFileNum - i, this.store.getStorefiles().size());
192 
193       // Let the next store file expired.
194       Thread.sleep(sleepTime);
195     }
196   }
197 
198   public void testLowestModificationTime() throws Exception {
199     Configuration conf = HBaseConfiguration.create();
200     FileSystem fs = FileSystem.get(conf);
201     // Initialize region
202     init(getName(), conf);
203     
204     int storeFileNum = 4;
205     for (int i = 1; i <= storeFileNum; i++) {
206       LOG.info("Adding some data for the store file #"+i);
207       this.store.add(new KeyValue(row, family, qf1, i, (byte[])null));
208       this.store.add(new KeyValue(row, family, qf2, i, (byte[])null));
209       this.store.add(new KeyValue(row, family, qf3, i, (byte[])null));
210       flush(i);
211     }
212     // after flush; check the lowest time stamp
213     long lowestTimeStampFromStore = 
214         Store.getLowestTimestamp(store.getStorefiles());
215     long lowestTimeStampFromFS = 
216       getLowestTimeStampFromFS(fs,store.getStorefiles());
217     assertEquals(lowestTimeStampFromStore,lowestTimeStampFromFS);
218     
219     // after compact; check the lowest time stamp
220     store.compact(store.requestCompaction());
221     lowestTimeStampFromStore = Store.getLowestTimestamp(store.getStorefiles());
222     lowestTimeStampFromFS = getLowestTimeStampFromFS(fs,store.getStorefiles());
223     assertEquals(lowestTimeStampFromStore,lowestTimeStampFromFS); 
224   }
225   
226   private static long getLowestTimeStampFromFS(FileSystem fs, 
227       final List<StoreFile> candidates) throws IOException {
228     long minTs = Long.MAX_VALUE;
229     if (candidates.isEmpty()) {
230       return minTs; 
231     }
232     Path[] p = new Path[candidates.size()];
233     for (int i = 0; i < candidates.size(); ++i) {
234       p[i] = candidates.get(i).getPath();
235     }
236     
237     FileStatus[] stats = fs.listStatus(p);
238     if (stats == null || stats.length == 0) {
239       return minTs;
240     }
241     for (FileStatus s : stats) {
242       minTs = Math.min(minTs, s.getModificationTime());
243     }
244     return minTs;
245   }
246 
247   //////////////////////////////////////////////////////////////////////////////
248   // Get tests
249   //////////////////////////////////////////////////////////////////////////////
250 
251   /**
252    * Test for hbase-1686.
253    * @throws IOException
254    */
255   public void testEmptyStoreFile() throws IOException {
256     init(this.getName());
257     // Write a store file.
258     this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null));
259     this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null));
260     flush(1);
261     // Now put in place an empty store file.  Its a little tricky.  Have to
262     // do manually with hacked in sequence id.
263     StoreFile f = this.store.getStorefiles().get(0);
264     Path storedir = f.getPath().getParent();
265     long seqid = f.getMaxSequenceId();
266     Configuration c = HBaseConfiguration.create();
267     FileSystem fs = FileSystem.get(c);
268     StoreFile.Writer w = new StoreFile.WriterBuilder(c, new CacheConfig(c),
269         fs, StoreFile.DEFAULT_BLOCKSIZE_SMALL)
270             .withOutputDir(storedir)
271             .build();
272     w.appendMetadata(seqid + 1, false);
273     w.close();
274     this.store.close();
275     // Reopen it... should pick up two files
276     this.store = new Store(storedir.getParent().getParent(),
277       this.store.getHRegion(),
278       this.store.getFamily(), fs, c);
279     System.out.println(this.store.getHRegionInfo().getEncodedName());
280     assertEquals(2, this.store.getStorefilesCount());
281 
282     result = HBaseTestingUtility.getFromStoreFile(store,
283         get.getRow(),
284         qualifiers);
285     assertEquals(1, result.size());
286   }
287 
288   /**
289    * Getting data from memstore only
290    * @throws IOException
291    */
292   public void testGet_FromMemStoreOnly() throws IOException {
293     init(this.getName());
294 
295     //Put data in memstore
296     this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null));
297     this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null));
298     this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null));
299     this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null));
300     this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null));
301     this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null));
302 
303     //Get
304     result = HBaseTestingUtility.getFromStoreFile(store,
305         get.getRow(), qualifiers);
306 
307     //Compare
308     assertCheck();
309   }
310 
311   /**
312    * Getting data from files only
313    * @throws IOException
314    */
315   public void testGet_FromFilesOnly() throws IOException {
316     init(this.getName());
317 
318     //Put data in memstore
319     this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null));
320     this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null));
321     //flush
322     flush(1);
323 
324     //Add more data
325     this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null));
326     this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null));
327     //flush
328     flush(2);
329 
330     //Add more data
331     this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null));
332     this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null));
333     //flush
334     flush(3);
335 
336     //Get
337     result = HBaseTestingUtility.getFromStoreFile(store,
338         get.getRow(),
339         qualifiers);
340     //this.store.get(get, qualifiers, result);
341 
342     //Need to sort the result since multiple files
343     Collections.sort(result, KeyValue.COMPARATOR);
344 
345     //Compare
346     assertCheck();
347   }
348 
349   /**
350    * Getting data from memstore and files
351    * @throws IOException
352    */
353   public void testGet_FromMemStoreAndFiles() throws IOException {
354     init(this.getName());
355 
356     //Put data in memstore
357     this.store.add(new KeyValue(row, family, qf1, 1, (byte[])null));
358     this.store.add(new KeyValue(row, family, qf2, 1, (byte[])null));
359     //flush
360     flush(1);
361 
362     //Add more data
363     this.store.add(new KeyValue(row, family, qf3, 1, (byte[])null));
364     this.store.add(new KeyValue(row, family, qf4, 1, (byte[])null));
365     //flush
366     flush(2);
367 
368     //Add more data
369     this.store.add(new KeyValue(row, family, qf5, 1, (byte[])null));
370     this.store.add(new KeyValue(row, family, qf6, 1, (byte[])null));
371 
372     //Get
373     result = HBaseTestingUtility.getFromStoreFile(store,
374         get.getRow(), qualifiers);
375 
376     //Need to sort the result since multiple files
377     Collections.sort(result, KeyValue.COMPARATOR);
378 
379     //Compare
380     assertCheck();
381   }
382 
383   private void flush(int storeFilessize) throws IOException{
384     this.store.snapshot();
385     flushStore(store, id++);
386     assertEquals(storeFilessize, this.store.getStorefiles().size());
387     assertEquals(0, this.store.memstore.kvset.size());
388   }
389 
390   private void assertCheck() {
391     assertEquals(expected.size(), result.size());
392     for(int i=0; i<expected.size(); i++) {
393       assertEquals(expected.get(i), result.get(i));
394     }
395   }
396 
397   //////////////////////////////////////////////////////////////////////////////
398   // IncrementColumnValue tests
399   //////////////////////////////////////////////////////////////////////////////
400   /*
401    * test the internal details of how ICV works, especially during a flush scenario.
402    */
403   public void testIncrementColumnValue_ICVDuringFlush()
404       throws IOException, InterruptedException {
405     init(this.getName());
406 
407     long oldValue = 1L;
408     long newValue = 3L;
409     this.store.add(new KeyValue(row, family, qf1,
410         System.currentTimeMillis(),
411         Bytes.toBytes(oldValue)));
412 
413     // snapshot the store.
414     this.store.snapshot();
415 
416     // add other things:
417     this.store.add(new KeyValue(row, family, qf2,
418         System.currentTimeMillis(),
419         Bytes.toBytes(oldValue)));
420 
421     // update during the snapshot.
422     long ret = this.store.updateColumnValue(row, family, qf1, newValue);
423 
424     // memstore should have grown by some amount.
425     assertTrue(ret > 0);
426 
427     // then flush.
428     flushStore(store, id++);
429     assertEquals(1, this.store.getStorefiles().size());
430     // from the one we inserted up there, and a new one
431     assertEquals(2, this.store.memstore.kvset.size());
432 
433     // how many key/values for this row are there?
434     Get get = new Get(row);
435     get.addColumn(family, qf1);
436     get.setMaxVersions(); // all versions.
437     List<KeyValue> results = new ArrayList<KeyValue>();
438 
439     results = HBaseTestingUtility.getFromStoreFile(store, get);
440     assertEquals(2, results.size());
441 
442     long ts1 = results.get(0).getTimestamp();
443     long ts2 = results.get(1).getTimestamp();
444 
445     assertTrue(ts1 > ts2);
446 
447     assertEquals(newValue, Bytes.toLong(results.get(0).getValue()));
448     assertEquals(oldValue, Bytes.toLong(results.get(1).getValue()));
449   }
450 
451   @Override
452   protected void tearDown() throws Exception {
453     super.tearDown();
454     EnvironmentEdgeManagerTestHelper.reset();
455   }
456 
457   public void testICV_negMemstoreSize()  throws IOException {
458       init(this.getName());
459 
460     long time = 100;
461     ManualEnvironmentEdge ee = new ManualEnvironmentEdge();
462     ee.setValue(time);
463     EnvironmentEdgeManagerTestHelper.injectEdge(ee);
464     long newValue = 3L;
465     long size = 0;
466 
467 
468     size += this.store.add(new KeyValue(Bytes.toBytes("200909091000"), family, qf1,
469         System.currentTimeMillis(),
470         Bytes.toBytes(newValue)));
471     size += this.store.add(new KeyValue(Bytes.toBytes("200909091200"), family, qf1,
472         System.currentTimeMillis(),
473         Bytes.toBytes(newValue)));
474     size += this.store.add(new KeyValue(Bytes.toBytes("200909091300"), family, qf1,
475         System.currentTimeMillis(),
476         Bytes.toBytes(newValue)));
477     size += this.store.add(new KeyValue(Bytes.toBytes("200909091400"), family, qf1,
478         System.currentTimeMillis(),
479         Bytes.toBytes(newValue)));
480     size += this.store.add(new KeyValue(Bytes.toBytes("200909091500"), family, qf1,
481         System.currentTimeMillis(),
482         Bytes.toBytes(newValue)));
483 
484 
485     for ( int i = 0 ; i < 10000 ; ++i) {
486       newValue++;
487 
488       long ret = this.store.updateColumnValue(row, family, qf1, newValue);
489       long ret2 = this.store.updateColumnValue(row2, family, qf1, newValue);
490 
491       if (ret != 0) System.out.println("ret: " + ret);
492       if (ret2 != 0) System.out.println("ret2: " + ret2);
493 
494       assertTrue("ret: " + ret, ret >= 0);
495       size += ret;
496       assertTrue("ret2: " + ret2, ret2 >= 0);
497       size += ret2;
498 
499 
500       if (i % 1000 == 0)
501         ee.setValue(++time);
502     }
503 
504     long computedSize=0;
505     for (KeyValue kv : this.store.memstore.kvset) {
506       long kvsize = this.store.memstore.heapSizeChange(kv, true);
507       //System.out.println(kv + " size= " + kvsize + " kvsize= " + kv.heapSize());
508       computedSize += kvsize;
509     }
510     assertEquals(computedSize, size);
511   }
512 
513   public void testIncrementColumnValue_SnapshotFlushCombo() throws Exception {
514     ManualEnvironmentEdge mee = new ManualEnvironmentEdge();
515     EnvironmentEdgeManagerTestHelper.injectEdge(mee);
516     init(this.getName());
517 
518     long oldValue = 1L;
519     long newValue = 3L;
520     this.store.add(new KeyValue(row, family, qf1,
521         EnvironmentEdgeManager.currentTimeMillis(),
522         Bytes.toBytes(oldValue)));
523 
524     // snapshot the store.
525     this.store.snapshot();
526 
527     // update during the snapshot, the exact same TS as the Put (lololol)
528     long ret = this.store.updateColumnValue(row, family, qf1, newValue);
529 
530     // memstore should have grown by some amount.
531     assertTrue(ret > 0);
532 
533     // then flush.
534     flushStore(store, id++);
535     assertEquals(1, this.store.getStorefiles().size());
536     assertEquals(1, this.store.memstore.kvset.size());
537 
538     // now increment again:
539     newValue += 1;
540     this.store.updateColumnValue(row, family, qf1, newValue);
541 
542     // at this point we have a TS=1 in snapshot, and a TS=2 in kvset, so increment again:
543     newValue += 1;
544     this.store.updateColumnValue(row, family, qf1, newValue);
545 
546     // the second TS should be TS=2 or higher., even though 'time=1' right now.
547 
548 
549     // how many key/values for this row are there?
550     Get get = new Get(row);
551     get.addColumn(family, qf1);
552     get.setMaxVersions(); // all versions.
553     List<KeyValue> results = new ArrayList<KeyValue>();
554 
555     results = HBaseTestingUtility.getFromStoreFile(store, get);
556     assertEquals(2, results.size());
557 
558     long ts1 = results.get(0).getTimestamp();
559     long ts2 = results.get(1).getTimestamp();
560 
561     assertTrue(ts1 > ts2);
562     assertEquals(newValue, Bytes.toLong(results.get(0).getValue()));
563     assertEquals(oldValue, Bytes.toLong(results.get(1).getValue()));
564 
565     mee.setValue(2); // time goes up slightly
566     newValue += 1;
567     this.store.updateColumnValue(row, family, qf1, newValue);
568 
569     results = HBaseTestingUtility.getFromStoreFile(store, get);
570     assertEquals(2, results.size());
571 
572     ts1 = results.get(0).getTimestamp();
573     ts2 = results.get(1).getTimestamp();
574 
575     assertTrue(ts1 > ts2);
576     assertEquals(newValue, Bytes.toLong(results.get(0).getValue()));
577     assertEquals(oldValue, Bytes.toLong(results.get(1).getValue()));
578   }
579 
580   public void testHandleErrorsInFlush() throws Exception {
581     LOG.info("Setting up a faulty file system that cannot write");
582 
583     final Configuration conf = HBaseConfiguration.create();
584     User user = User.createUserForTesting(conf,
585         "testhandleerrorsinflush", new String[]{"foo"});
586     // Inject our faulty LocalFileSystem
587     conf.setClass("fs.file.impl", FaultyFileSystem.class,
588         FileSystem.class);
589     user.runAs(new PrivilegedExceptionAction<Object>() {
590       public Object run() throws Exception {
591         // Make sure it worked (above is sensitive to caching details in hadoop core)
592         FileSystem fs = FileSystem.get(conf);
593         assertEquals(FaultyFileSystem.class, fs.getClass());
594 
595         // Initialize region
596         init(getName(), conf);
597 
598         LOG.info("Adding some data");
599         store.add(new KeyValue(row, family, qf1, 1, (byte[])null));
600         store.add(new KeyValue(row, family, qf2, 1, (byte[])null));
601         store.add(new KeyValue(row, family, qf3, 1, (byte[])null));
602 
603         LOG.info("Before flush, we should have no files");
604         FileStatus[] files = fs.listStatus(store.getHomedir());
605         Path[] paths = FileUtil.stat2Paths(files);
606         System.err.println("Got paths: " + Joiner.on(",").join(paths));
607         assertEquals(0, paths.length);
608 
609         //flush
610         try {
611           LOG.info("Flushing");
612           flush(1);
613           fail("Didn't bubble up IOE!");
614         } catch (IOException ioe) {
615           assertTrue(ioe.getMessage().contains("Fault injected"));
616         }
617 
618         LOG.info("After failed flush, we should still have no files!");
619         files = fs.listStatus(store.getHomedir());
620         paths = FileUtil.stat2Paths(files);
621         System.err.println("Got paths: " + Joiner.on(",").join(paths));
622         assertEquals(0, paths.length);
623         return null;
624       }
625     });
626   }
627 
628 
629   static class FaultyFileSystem extends FilterFileSystem {
630     List<SoftReference<FaultyOutputStream>> outStreams =
631       new ArrayList<SoftReference<FaultyOutputStream>>();
632     private long faultPos = 200;
633 
634     public FaultyFileSystem() {
635       super(new LocalFileSystem());
636       System.err.println("Creating faulty!");
637     }
638 
639     @Override
640     public FSDataOutputStream create(Path p) throws IOException {
641       return new FaultyOutputStream(super.create(p), faultPos);
642     }
643 
644     @Override
645     public FSDataOutputStream create(Path f, FsPermission permission,
646         boolean overwrite, int bufferSize, short replication, long blockSize,
647         Progressable progress) throws IOException {
648       return new FaultyOutputStream(super.create(f, permission,
649           overwrite, bufferSize, replication, blockSize, progress), faultPos);
650     }
651 
652     public FSDataOutputStream createNonRecursive(Path f, boolean overwrite,
653         int bufferSize, short replication, long blockSize, Progressable progress)
654     throws IOException {
655       // Fake it.  Call create instead.  The default implementation throws an IOE
656       // that this is not supported.
657       return create(f, overwrite, bufferSize, replication, blockSize, progress);
658     }
659   }
660 
661   static class FaultyOutputStream extends FSDataOutputStream {
662     volatile long faultPos = Long.MAX_VALUE;
663 
664     public FaultyOutputStream(FSDataOutputStream out,
665         long faultPos) throws IOException {
666       super(out, null);
667       this.faultPos = faultPos;
668     }
669 
670     @Override
671     public void write(byte[] buf, int offset, int length) throws IOException {
672       System.err.println("faulty stream write at pos " + getPos());
673       injectFault();
674       super.write(buf, offset, length);
675     }
676 
677     private void injectFault() throws IOException {
678       if (getPos() >= faultPos) {
679         throw new IOException("Fault injected");
680       }
681     }
682   }
683 
684 
685 
686   private static void flushStore(Store store, long id) throws IOException {
687     StoreFlusher storeFlusher = store.getStoreFlusher(id);
688     storeFlusher.prepare();
689     storeFlusher.flushCache(Mockito.mock(MonitoredTask.class));
690     storeFlusher.commit(Mockito.mock(MonitoredTask.class));
691   }
692 
693 
694 
695   /**
696    * Generate a list of KeyValues for testing based on given parameters
697    * @param timestamps
698    * @param numRows
699    * @param qualifier
700    * @param family
701    * @return
702    */
703   List<KeyValue> getKeyValueSet(long[] timestamps, int numRows,
704       byte[] qualifier, byte[] family) {
705     List<KeyValue> kvList = new ArrayList<KeyValue>();
706     for (int i=1;i<=numRows;i++) {
707       byte[] b = Bytes.toBytes(i);
708       for (long timestamp: timestamps) {
709         kvList.add(new KeyValue(b, family, qualifier, timestamp, b));
710       }
711     }
712     return kvList;
713   }
714 
715   /**
716    * Test to ensure correctness when using Stores with multiple timestamps
717    * @throws IOException
718    */
719   public void testMultipleTimestamps() throws IOException {
720     int numRows = 1;
721     long[] timestamps1 = new long[] {1,5,10,20};
722     long[] timestamps2 = new long[] {30,80};
723 
724     init(this.getName());
725 
726     List<KeyValue> kvList1 = getKeyValueSet(timestamps1,numRows, qf1, family);
727     for (KeyValue kv : kvList1) {
728       this.store.add(kv);
729     }
730 
731     this.store.snapshot();
732     flushStore(store, id++);
733 
734     List<KeyValue> kvList2 = getKeyValueSet(timestamps2,numRows, qf1, family);
735     for(KeyValue kv : kvList2) {
736       this.store.add(kv);
737     }
738 
739     List<KeyValue> result;
740     Get get = new Get(Bytes.toBytes(1));
741     get.addColumn(family,qf1);
742 
743     get.setTimeRange(0,15);
744     result = HBaseTestingUtility.getFromStoreFile(store, get);
745     assertTrue(result.size()>0);
746 
747     get.setTimeRange(40,90);
748     result = HBaseTestingUtility.getFromStoreFile(store, get);
749     assertTrue(result.size()>0);
750 
751     get.setTimeRange(10,45);
752     result = HBaseTestingUtility.getFromStoreFile(store, get);
753     assertTrue(result.size()>0);
754 
755     get.setTimeRange(80,145);
756     result = HBaseTestingUtility.getFromStoreFile(store, get);
757     assertTrue(result.size()>0);
758 
759     get.setTimeRange(1,2);
760     result = HBaseTestingUtility.getFromStoreFile(store, get);
761     assertTrue(result.size()>0);
762 
763     get.setTimeRange(90,200);
764     result = HBaseTestingUtility.getFromStoreFile(store, get);
765     assertTrue(result.size()==0);
766   }
767 
768   /**
769    * Test for HBASE-3492 - Test split on empty colfam (no store files).
770    *
771    * @throws IOException When the IO operations fail.
772    */
773   public void testSplitWithEmptyColFam() throws IOException {
774     init(this.getName());
775     assertNull(store.getSplitPoint());
776     store.getHRegion().forceSplit(null);
777     assertNull(store.getSplitPoint());
778     store.getHRegion().clearSplit_TESTS_ONLY();
779   }
780 
781   @org.junit.Rule
782   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
783     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
784 }
785