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