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