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