1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.regionserver;
21  
22  
23  import java.io.IOException;
24  import java.util.ArrayList;
25  import java.util.Arrays;
26  import java.util.HashMap;
27  import java.util.List;
28  import java.util.Map;
29  import java.util.TreeMap;
30  import java.util.concurrent.atomic.AtomicBoolean;
31  import java.util.concurrent.atomic.AtomicInteger;
32  import java.util.concurrent.atomic.AtomicReference;
33  
34  import org.apache.commons.logging.Log;
35  import org.apache.commons.logging.LogFactory;
36  import org.apache.hadoop.conf.Configuration;
37  import org.apache.hadoop.fs.FSDataOutputStream;
38  import org.apache.hadoop.fs.FileSystem;
39  import org.apache.hadoop.fs.Path;
40  import org.apache.hadoop.hbase.DoNotRetryIOException;
41  import org.apache.hadoop.hbase.HBaseConfiguration;
42  import org.apache.hadoop.hbase.HBaseTestCase;
43  import org.apache.hadoop.hbase.HBaseTestingUtility;
44  import org.apache.hadoop.hbase.HColumnDescriptor;
45  import org.apache.hadoop.hbase.HConstants;
46  import org.apache.hadoop.hbase.HDFSBlocksDistribution;
47  import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
48  import org.apache.hadoop.hbase.HRegionInfo;
49  import org.apache.hadoop.hbase.HTableDescriptor;
50  import org.apache.hadoop.hbase.KeyValue;
51  import org.apache.hadoop.hbase.MultithreadedTestUtil;
52  import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
53  import org.apache.hadoop.hbase.MiniHBaseCluster;
54  import org.apache.hadoop.hbase.client.Delete;
55  import org.apache.hadoop.hbase.client.Get;
56  import org.apache.hadoop.hbase.client.HTable;
57  import org.apache.hadoop.hbase.client.Put;
58  import org.apache.hadoop.hbase.client.Result;
59  import org.apache.hadoop.hbase.client.Scan;
60  import org.apache.hadoop.hbase.filter.BinaryComparator;
61  import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
62  import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
63  import org.apache.hadoop.hbase.filter.Filter;
64  import org.apache.hadoop.hbase.filter.FilterList;
65  import org.apache.hadoop.hbase.filter.NullComparator;
66  import org.apache.hadoop.hbase.filter.PrefixFilter;
67  import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
68  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
69  import org.apache.hadoop.hbase.monitoring.TaskMonitor;
70  import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
71  import org.apache.hadoop.hbase.regionserver.wal.HLog;
72  import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
73  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
74  import org.apache.hadoop.hbase.util.Bytes;
75  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
76  import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
77  import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
78  import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
79  import org.apache.hadoop.hbase.util.Pair;
80  import org.apache.hadoop.hbase.util.PairOfSameType;
81  import org.apache.hadoop.hbase.util.Threads;
82  import org.junit.Test;
83  
84  import com.google.common.collect.Lists;
85  
86  
87  /**
88   * Basic stand-alone testing of HRegion.
89   *
90   * A lot of the meta information for an HRegion now lives inside other
91   * HRegions or in the HBaseMaster, so only basic testing is possible.
92   */
93  public class TestHRegion extends HBaseTestCase {
94    static final Log LOG = LogFactory.getLog(TestHRegion.class);
95  
96    HRegion region = null;
97    private HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
98    private final String DIR = TEST_UTIL.getDataTestDir("TestHRegion").toString();
99  
100   private final int MAX_VERSIONS = 2;
101 
102   // Test names
103   protected final byte[] tableName = Bytes.toBytes("testtable");;
104   protected final byte[] qual1 = Bytes.toBytes("qual1");
105   protected final byte[] qual2 = Bytes.toBytes("qual2");
106   protected final byte[] qual3 = Bytes.toBytes("qual3");
107   protected final byte[] value1 = Bytes.toBytes("value1");
108   protected final byte[] value2 = Bytes.toBytes("value2");
109   protected final byte [] row = Bytes.toBytes("rowA");
110   protected final byte [] row2 = Bytes.toBytes("rowB");
111 
112 
113   /**
114    * @see org.apache.hadoop.hbase.HBaseTestCase#setUp()
115    */
116   @Override
117   protected void setUp() throws Exception {
118     super.setUp();
119   }
120 
121   @Override
122   protected void tearDown() throws Exception {
123     super.tearDown();
124     EnvironmentEdgeManagerTestHelper.reset();
125   }
126 
127   //////////////////////////////////////////////////////////////////////////////
128   // New tests that doesn't spin up a mini cluster but rather just test the
129   // individual code pieces in the HRegion. Putting files locally in
130   // /tmp/testtable
131   //////////////////////////////////////////////////////////////////////////////
132 
133   public void testGetWhileRegionClose() throws IOException {
134     Configuration hc = initSplit();
135     int numRows = 100;
136     byte [][] families = {fam1, fam2, fam3};
137 
138     //Setting up region
139     String method = this.getName();
140     initHRegion(tableName, method, hc, families);
141 
142     // Put data in region
143     final int startRow = 100;
144     putData(startRow, numRows, qual1, families);
145     putData(startRow, numRows, qual2, families);
146     putData(startRow, numRows, qual3, families);
147     // this.region.flushcache();
148     final AtomicBoolean done = new AtomicBoolean(false);
149     final AtomicInteger gets = new AtomicInteger(0);
150     GetTillDoneOrException [] threads = new GetTillDoneOrException[10];
151     try {
152       // Set ten threads running concurrently getting from the region.
153       for (int i = 0; i < threads.length / 2; i++) {
154         threads[i] = new GetTillDoneOrException(i, Bytes.toBytes("" + startRow),
155           done, gets);
156         threads[i].setDaemon(true);
157         threads[i].start();
158       }
159       // Artificially make the condition by setting closing flag explicitly.
160       // I can't make the issue happen with a call to region.close().
161       this.region.closing.set(true);
162       for (int i = threads.length / 2; i < threads.length; i++) {
163         threads[i] = new GetTillDoneOrException(i, Bytes.toBytes("" + startRow),
164           done, gets);
165         threads[i].setDaemon(true);
166         threads[i].start();
167       }
168     } finally {
169       if (this.region != null) {
170         this.region.close();
171         this.region.getLog().closeAndDelete();
172       }
173     }
174     done.set(true);
175     for (GetTillDoneOrException t: threads) {
176       try {
177         t.join();
178       } catch (InterruptedException e) {
179         e.printStackTrace();
180       }
181       if (t.e != null) {
182         LOG.info("Exception=" + t.e);
183         assertFalse("Found a NPE in " + t.getName(),
184           t.e instanceof NullPointerException);
185       }
186     }
187   }
188 
189   /*
190    * Thread that does get on single row until 'done' flag is flipped.  If an
191    * exception causes us to fail, it records it.
192    */
193   class GetTillDoneOrException extends Thread {
194     private final Get g;
195     private final AtomicBoolean done;
196     private final AtomicInteger count;
197     private Exception e;
198 
199     GetTillDoneOrException(final int i, final byte[] r, final AtomicBoolean d,
200         final AtomicInteger c) {
201       super("getter." + i);
202       this.g = new Get(r);
203       this.done = d;
204       this.count = c;
205     }
206 
207     @Override
208     public void run() {
209       while (!this.done.get()) {
210         try {
211           assertTrue(region.get(g, null).size() > 0);
212           this.count.incrementAndGet();
213         } catch (Exception e) {
214           this.e = e;
215           break;
216         }
217       }
218     }
219   }
220 
221   /*
222    * An involved filter test.  Has multiple column families and deletes in mix.
223    */
224   public void testWeirdCacheBehaviour() throws Exception {
225     byte[] TABLE = Bytes.toBytes("testWeirdCacheBehaviour");
226     byte[][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"),
227         Bytes.toBytes("trans-type"), Bytes.toBytes("trans-date"),
228         Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") };
229     initHRegion(TABLE, getName(), FAMILIES);
230     String value = "this is the value";
231     String value2 = "this is some other value";
232     String keyPrefix1 = "prefix1"; // UUID.randomUUID().toString();
233     String keyPrefix2 = "prefix2"; // UUID.randomUUID().toString();
234     String keyPrefix3 = "prefix3"; // UUID.randomUUID().toString();
235     putRows(this.region, 3, value, keyPrefix1);
236     putRows(this.region, 3, value, keyPrefix2);
237     putRows(this.region, 3, value, keyPrefix3);
238     // this.region.flushCommits();
239     putRows(this.region, 3, value2, keyPrefix1);
240     putRows(this.region, 3, value2, keyPrefix2);
241     putRows(this.region, 3, value2, keyPrefix3);
242     System.out.println("Checking values for key: " + keyPrefix1);
243     assertEquals("Got back incorrect number of rows from scan", 3,
244       getNumberOfRows(keyPrefix1, value2, this.region));
245     System.out.println("Checking values for key: " + keyPrefix2);
246     assertEquals("Got back incorrect number of rows from scan", 3,
247       getNumberOfRows(keyPrefix2, value2, this.region));
248     System.out.println("Checking values for key: " + keyPrefix3);
249     assertEquals("Got back incorrect number of rows from scan", 3,
250       getNumberOfRows(keyPrefix3, value2, this.region));
251     deleteColumns(this.region, value2, keyPrefix1);
252     deleteColumns(this.region, value2, keyPrefix2);
253     deleteColumns(this.region, value2, keyPrefix3);
254     System.out.println("Starting important checks.....");
255     assertEquals("Got back incorrect number of rows from scan: " + keyPrefix1,
256       0, getNumberOfRows(keyPrefix1, value2, this.region));
257     assertEquals("Got back incorrect number of rows from scan: " + keyPrefix2,
258       0, getNumberOfRows(keyPrefix2, value2, this.region));
259     assertEquals("Got back incorrect number of rows from scan: " + keyPrefix3,
260       0, getNumberOfRows(keyPrefix3, value2, this.region));
261   }
262 
263   private void deleteColumns(HRegion r, String value, String keyPrefix)
264   throws IOException {
265     InternalScanner scanner = buildScanner(keyPrefix, value, r);
266     int count = 0;
267     boolean more = false;
268     List<KeyValue> results = new ArrayList<KeyValue>();
269     do {
270       more = scanner.next(results);
271       if (results != null && !results.isEmpty())
272         count++;
273       else
274         break;
275       Delete delete = new Delete(results.get(0).getRow());
276       delete.deleteColumn(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"));
277       r.delete(delete, null, false);
278       results.clear();
279     } while (more);
280     assertEquals("Did not perform correct number of deletes", 3, count);
281   }
282 
283   private int getNumberOfRows(String keyPrefix, String value, HRegion r) throws Exception {
284     InternalScanner resultScanner = buildScanner(keyPrefix, value, r);
285     int numberOfResults = 0;
286     List<KeyValue> results = new ArrayList<KeyValue>();
287     boolean more = false;
288     do {
289       more = resultScanner.next(results);
290       if (results != null && !results.isEmpty()) numberOfResults++;
291       else break;
292       for (KeyValue kv: results) {
293         System.out.println("kv=" + kv.toString() + ", " + Bytes.toString(kv.getValue()));
294       }
295       results.clear();
296     } while(more);
297     return numberOfResults;
298   }
299 
300   private InternalScanner buildScanner(String keyPrefix, String value, HRegion r)
301   throws IOException {
302     // Defaults FilterList.Operator.MUST_PASS_ALL.
303     FilterList allFilters = new FilterList();
304     allFilters.addFilter(new PrefixFilter(Bytes.toBytes(keyPrefix)));
305     // Only return rows where this column value exists in the row.
306     SingleColumnValueFilter filter =
307       new SingleColumnValueFilter(Bytes.toBytes("trans-tags"),
308         Bytes.toBytes("qual2"), CompareOp.EQUAL, Bytes.toBytes(value));
309     filter.setFilterIfMissing(true);
310     allFilters.addFilter(filter);
311     Scan scan = new Scan();
312     scan.addFamily(Bytes.toBytes("trans-blob"));
313     scan.addFamily(Bytes.toBytes("trans-type"));
314     scan.addFamily(Bytes.toBytes("trans-date"));
315     scan.addFamily(Bytes.toBytes("trans-tags"));
316     scan.addFamily(Bytes.toBytes("trans-group"));
317     scan.setFilter(allFilters);
318     return r.getScanner(scan);
319   }
320 
321   private void putRows(HRegion r, int numRows, String value, String key)
322   throws IOException {
323     for (int i = 0; i < numRows; i++) {
324       String row = key + "_" + i/* UUID.randomUUID().toString() */;
325       System.out.println(String.format("Saving row: %s, with value %s", row,
326         value));
327       Put put = new Put(Bytes.toBytes(row));
328       put.setWriteToWAL(false);
329       put.add(Bytes.toBytes("trans-blob"), null,
330         Bytes.toBytes("value for blob"));
331       put.add(Bytes.toBytes("trans-type"), null, Bytes.toBytes("statement"));
332       put.add(Bytes.toBytes("trans-date"), null,
333         Bytes.toBytes("20090921010101999"));
334       put.add(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"),
335         Bytes.toBytes(value));
336       put.add(Bytes.toBytes("trans-group"), null,
337         Bytes.toBytes("adhocTransactionGroupId"));
338       r.put(put);
339     }
340   }
341 
342   public void testFamilyWithAndWithoutColon() throws Exception {
343     byte [] b = Bytes.toBytes(getName());
344     byte [] cf = Bytes.toBytes("cf");
345     initHRegion(b, getName(), cf);
346     Put p = new Put(b);
347     byte [] cfwithcolon = Bytes.toBytes("cf:");
348     p.add(cfwithcolon, cfwithcolon, cfwithcolon);
349     boolean exception = false;
350     try {
351       this.region.put(p);
352     } catch (NoSuchColumnFamilyException e) {
353       exception = true;
354     }
355     assertTrue(exception);
356   }
357 
358   @SuppressWarnings("unchecked")
359   public void testBatchPut() throws Exception {
360     byte[] b = Bytes.toBytes(getName());
361     byte[] cf = Bytes.toBytes("cf");
362     byte[] qual = Bytes.toBytes("qual");
363     byte[] val = Bytes.toBytes("val");
364     initHRegion(b, getName(), cf);
365 
366     HLog.getSyncOps(); // clear counter from prior tests
367     assertEquals(0, HLog.getSyncOps());
368 
369     LOG.info("First a batch put with all valid puts");
370     final Put[] puts = new Put[10];
371     for (int i = 0; i < 10; i++) {
372       puts[i] = new Put(Bytes.toBytes("row_" + i));
373       puts[i].add(cf, qual, val);
374     }
375 
376     OperationStatus[] codes = this.region.put(puts);
377     assertEquals(10, codes.length);
378     for (int i = 0; i < 10; i++) {
379       assertEquals(OperationStatusCode.SUCCESS, codes[i]
380           .getOperationStatusCode());
381     }
382     assertEquals(1, HLog.getSyncOps());
383 
384     LOG.info("Next a batch put with one invalid family");
385     puts[5].add(Bytes.toBytes("BAD_CF"), qual, val);
386     codes = this.region.put(puts);
387     assertEquals(10, codes.length);
388     for (int i = 0; i < 10; i++) {
389       assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
390         OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
391     }
392     assertEquals(1, HLog.getSyncOps());
393 
394     LOG.info("Next a batch put that has to break into two batches to avoid a lock");
395     Integer lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
396 
397     MultithreadedTestUtil.TestContext ctx =
398       new MultithreadedTestUtil.TestContext(HBaseConfiguration.create());
399     final AtomicReference<OperationStatus[]> retFromThread =
400       new AtomicReference<OperationStatus[]>();
401     TestThread putter = new TestThread(ctx) {
402       @Override
403       public void doWork() throws IOException {
404         retFromThread.set(region.put(puts));
405       }
406     };
407     LOG.info("...starting put thread while holding lock");
408     ctx.addThread(putter);
409     ctx.startThreads();
410 
411     LOG.info("...waiting for put thread to sync first time");
412     long startWait = System.currentTimeMillis();
413     while (HLog.getSyncOps() == 0) {
414       Thread.sleep(100);
415       if (System.currentTimeMillis() - startWait > 10000) {
416         fail("Timed out waiting for thread to sync first minibatch");
417       }
418     }
419     LOG.info("...releasing row lock, which should let put thread continue");
420     region.releaseRowLock(lockedRow);
421     LOG.info("...joining on thread");
422     ctx.stop();
423     LOG.info("...checking that next batch was synced");
424     assertEquals(1, HLog.getSyncOps());
425     codes = retFromThread.get();
426     for (int i = 0; i < 10; i++) {
427       assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
428         OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
429     }
430 
431     LOG.info("Nexta, a batch put which uses an already-held lock");
432     lockedRow = region.obtainRowLock(Bytes.toBytes("row_2"));
433     LOG.info("...obtained row lock");
434     List<Pair<Put, Integer>> putsAndLocks = Lists.newArrayList();
435     for (int i = 0; i < 10; i++) {
436       Pair<Put, Integer> pair = new Pair<Put, Integer>(puts[i], null);
437       if (i == 2) pair.setSecond(lockedRow);
438       putsAndLocks.add(pair);
439     }
440 
441     codes = region.put(putsAndLocks.toArray(new Pair[0]));
442     LOG.info("...performed put");
443     for (int i = 0; i < 10; i++) {
444       assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY :
445         OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
446     }
447     // Make sure we didn't do an extra batch
448     assertEquals(1, HLog.getSyncOps());
449 
450     // Make sure we still hold lock
451     assertTrue(region.isRowLocked(lockedRow));
452     LOG.info("...releasing lock");
453     region.releaseRowLock(lockedRow);
454   }
455 
456   //////////////////////////////////////////////////////////////////////////////
457   // checkAndMutate tests
458   //////////////////////////////////////////////////////////////////////////////
459   public void testCheckAndMutate_WithEmptyRowValue() throws IOException {
460     byte [] tableName = Bytes.toBytes("testtable");
461     byte [] row1 = Bytes.toBytes("row1");
462     byte [] fam1 = Bytes.toBytes("fam1");
463     byte [] qf1  = Bytes.toBytes("qualifier");
464     byte [] emptyVal  = new byte[] {};
465     byte [] val1  = Bytes.toBytes("value1");
466     byte [] val2  = Bytes.toBytes("value2");
467     Integer lockId = null;
468 
469     //Setting up region
470     String method = this.getName();
471     initHRegion(tableName, method, fam1);
472     //Putting data in key
473     Put put = new Put(row1);
474     put.add(fam1, qf1, val1);
475 
476     //checkAndPut with correct value
477     boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
478         new BinaryComparator(emptyVal), put, lockId, true);
479     assertTrue(res);
480 
481     // not empty anymore
482     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
483         new BinaryComparator(emptyVal), put, lockId, true);
484     assertFalse(res);
485 
486     Delete delete = new Delete(row1);
487     delete.deleteColumn(fam1, qf1);
488     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
489         new BinaryComparator(emptyVal), delete, lockId, true);
490     assertFalse(res);
491 
492     put = new Put(row1);
493     put.add(fam1, qf1, val2);
494     //checkAndPut with correct value
495     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
496         new BinaryComparator(val1), put, lockId, true);
497     assertTrue(res);
498 
499     //checkAndDelete with correct value
500     delete = new Delete(row1);
501     delete.deleteColumn(fam1, qf1);
502     delete.deleteColumn(fam1, qf1);
503     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
504         new BinaryComparator(val2), delete, lockId, true);
505     assertTrue(res);
506 
507     delete = new Delete(row1);
508     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
509         new BinaryComparator(emptyVal), delete, lockId, true);
510     assertTrue(res);
511 
512     //checkAndPut looking for a null value
513     put = new Put(row1);
514     put.add(fam1, qf1, val1);
515 
516     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
517         new NullComparator(), put, lockId, true);
518     assertTrue(res);
519 
520   }
521 
522   public void testCheckAndMutate_WithWrongValue() throws IOException{
523     byte [] tableName = Bytes.toBytes("testtable");
524     byte [] row1 = Bytes.toBytes("row1");
525     byte [] fam1 = Bytes.toBytes("fam1");
526     byte [] qf1  = Bytes.toBytes("qualifier");
527     byte [] val1  = Bytes.toBytes("value1");
528     byte [] val2  = Bytes.toBytes("value2");
529     Integer lockId = null;
530 
531     //Setting up region
532     String method = this.getName();
533     initHRegion(tableName, method, fam1);
534 
535     //Putting data in key
536     Put put = new Put(row1);
537     put.add(fam1, qf1, val1);
538     region.put(put);
539 
540     //checkAndPut with wrong value
541     boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
542       new BinaryComparator(val2), put, lockId, true);
543     assertEquals(false, res);
544 
545     //checkAndDelete with wrong value
546     Delete delete = new Delete(row1);
547     delete.deleteFamily(fam1);
548     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
549       new BinaryComparator(val2), delete, lockId, true);
550     assertEquals(false, res);
551   }
552 
553   public void testCheckAndMutate_WithCorrectValue() throws IOException{
554     byte [] tableName = Bytes.toBytes("testtable");
555     byte [] row1 = Bytes.toBytes("row1");
556     byte [] fam1 = Bytes.toBytes("fam1");
557     byte [] qf1  = Bytes.toBytes("qualifier");
558     byte [] val1  = Bytes.toBytes("value1");
559     Integer lockId = null;
560 
561     //Setting up region
562     String method = this.getName();
563     initHRegion(tableName, method, fam1);
564 
565     //Putting data in key
566     Put put = new Put(row1);
567     put.add(fam1, qf1, val1);
568     region.put(put);
569 
570     //checkAndPut with correct value
571     boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
572       new BinaryComparator(val1), put, lockId, true);
573     assertEquals(true, res);
574 
575     //checkAndDelete with correct value
576     Delete delete = new Delete(row1);
577     delete.deleteColumn(fam1, qf1);
578     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
579       new BinaryComparator(val1), put, lockId, true);
580     assertEquals(true, res);
581   }
582 
583   public void testCheckAndPut_ThatPutWasWritten() throws IOException{
584     byte [] tableName = Bytes.toBytes("testtable");
585     byte [] row1 = Bytes.toBytes("row1");
586     byte [] fam1 = Bytes.toBytes("fam1");
587     byte [] fam2 = Bytes.toBytes("fam2");
588     byte [] qf1  = Bytes.toBytes("qualifier");
589     byte [] val1  = Bytes.toBytes("value1");
590     byte [] val2  = Bytes.toBytes("value2");
591     Integer lockId = null;
592 
593     byte [][] families = {fam1, fam2};
594 
595     //Setting up region
596     String method = this.getName();
597     initHRegion(tableName, method, families);
598 
599     //Putting data in the key to check
600     Put put = new Put(row1);
601     put.add(fam1, qf1, val1);
602     region.put(put);
603 
604     //Creating put to add
605     long ts = System.currentTimeMillis();
606     KeyValue kv = new KeyValue(row1, fam2, qf1, ts, KeyValue.Type.Put, val2);
607     put = new Put(row1);
608     put.add(kv);
609 
610     //checkAndPut with wrong value
611     Store store = region.getStore(fam1);
612     store.memstore.kvset.size();
613 
614     boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
615       new BinaryComparator(val1), put, lockId, true);
616     assertEquals(true, res);
617     store.memstore.kvset.size();
618 
619     Get get = new Get(row1);
620     get.addColumn(fam2, qf1);
621     KeyValue [] actual = region.get(get, null).raw();
622 
623     KeyValue [] expected = {kv};
624 
625     assertEquals(expected.length, actual.length);
626     for(int i=0; i<actual.length; i++) {
627       assertEquals(expected[i], actual[i]);
628     }
629 
630   }
631 
632   public void testCheckAndPut_wrongRowInPut() throws IOException {
633     initHRegion(tableName, this.getName(), COLUMNS);
634 
635     Put put = new Put(row2);
636     put.add(fam1, qual1, value1);
637     try {
638     boolean res = region.checkAndMutate(row, fam1, qual1, CompareOp.EQUAL,
639         new BinaryComparator(value2), put, null, false);
640       fail();
641     } catch (DoNotRetryIOException expected) {
642       // expected exception.
643     }
644   }
645 
646   public void testCheckAndDelete_ThatDeleteWasWritten() throws IOException{
647     byte [] tableName = Bytes.toBytes("testtable");
648     byte [] row1 = Bytes.toBytes("row1");
649     byte [] fam1 = Bytes.toBytes("fam1");
650     byte [] fam2 = Bytes.toBytes("fam2");
651     byte [] qf1  = Bytes.toBytes("qualifier1");
652     byte [] qf2  = Bytes.toBytes("qualifier2");
653     byte [] qf3  = Bytes.toBytes("qualifier3");
654     byte [] val1  = Bytes.toBytes("value1");
655     byte [] val2  = Bytes.toBytes("value2");
656     byte [] val3  = Bytes.toBytes("value3");
657     byte[] emptyVal = new byte[] { };
658     Integer lockId = null;
659 
660     byte [][] families = {fam1, fam2};
661 
662     //Setting up region
663     String method = this.getName();
664     initHRegion(tableName, method, families);
665 
666     //Put content
667     Put put = new Put(row1);
668     put.add(fam1, qf1, val1);
669     region.put(put);
670     Threads.sleep(2);
671 
672     put = new Put(row1);
673     put.add(fam1, qf1, val2);
674     put.add(fam2, qf1, val3);
675     put.add(fam2, qf2, val2);
676     put.add(fam2, qf3, val1);
677     put.add(fam1, qf3, val1);
678     region.put(put);
679 
680     //Multi-column delete
681     Delete delete = new Delete(row1);
682     delete.deleteColumn(fam1, qf1);
683     delete.deleteColumn(fam2, qf1);
684     delete.deleteColumn(fam1, qf3);
685     boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
686       new BinaryComparator(val2), delete, lockId, true);
687     assertEquals(true, res);
688 
689     Get get = new Get(row1);
690     get.addColumn(fam1, qf1);
691     get.addColumn(fam1, qf3);
692     get.addColumn(fam2, qf2);
693     Result r = region.get(get, null);
694     assertEquals(2, r.size());
695     assertEquals(val1, r.getValue(fam1, qf1));
696     assertEquals(val2, r.getValue(fam2, qf2));
697 
698     //Family delete
699     delete = new Delete(row1);
700     delete.deleteFamily(fam2);
701     res = region.checkAndMutate(row1, fam2, qf1, CompareOp.EQUAL,
702       new BinaryComparator(emptyVal), delete, lockId, true);
703     assertEquals(true, res);
704 
705     get = new Get(row1);
706     r = region.get(get, null);
707     assertEquals(1, r.size());
708     assertEquals(val1, r.getValue(fam1, qf1));
709 
710     //Row delete
711     delete = new Delete(row1);
712     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL,
713       new BinaryComparator(val1), delete, lockId, true);
714     assertEquals(true, res);
715     get = new Get(row1);
716     r = region.get(get, null);
717     assertEquals(0, r.size());
718   }
719 
720   //////////////////////////////////////////////////////////////////////////////
721   // Delete tests
722   //////////////////////////////////////////////////////////////////////////////
723   public void testDelete_multiDeleteColumn() throws IOException {
724     byte [] tableName = Bytes.toBytes("testtable");
725     byte [] row1 = Bytes.toBytes("row1");
726     byte [] fam1 = Bytes.toBytes("fam1");
727     byte [] qual = Bytes.toBytes("qualifier");
728     byte [] value = Bytes.toBytes("value");
729 
730     Put put = new Put(row1);
731     put.add(fam1, qual, 1, value);
732     put.add(fam1, qual, 2, value);
733 
734     String method = this.getName();
735     initHRegion(tableName, method, fam1);
736 
737     region.put(put);
738 
739     // We do support deleting more than 1 'latest' version
740     Delete delete = new Delete(row1);
741     delete.deleteColumn(fam1, qual);
742     delete.deleteColumn(fam1, qual);
743     region.delete(delete, null, false);
744 
745     Get get = new Get(row1);
746     get.addFamily(fam1);
747     Result r = region.get(get, null);
748     assertEquals(0, r.size());
749   }
750 
751   public void testDelete_CheckFamily() throws IOException {
752     byte [] tableName = Bytes.toBytes("testtable");
753     byte [] row1 = Bytes.toBytes("row1");
754     byte [] fam1 = Bytes.toBytes("fam1");
755     byte [] fam2 = Bytes.toBytes("fam2");
756     byte [] fam3 = Bytes.toBytes("fam3");
757     byte [] fam4 = Bytes.toBytes("fam4");
758 
759     //Setting up region
760     String method = this.getName();
761     initHRegion(tableName, method, fam1, fam2, fam3);
762 
763     List<KeyValue> kvs  = new ArrayList<KeyValue>();
764     kvs.add(new KeyValue(row1, fam4, null, null));
765 
766 
767     //testing existing family
768     byte [] family = fam2;
769     try {
770       Map<byte[], List<KeyValue>> deleteMap = new HashMap<byte[], List<KeyValue>>();
771       deleteMap.put(family, kvs);
772       region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
773     } catch (Exception e) {
774       assertTrue("Family " +new String(family)+ " does not exist", false);
775     }
776 
777     //testing non existing family
778     boolean ok = false;
779     family = fam4;
780     try {
781       Map<byte[], List<KeyValue>> deleteMap = new HashMap<byte[], List<KeyValue>>();
782       deleteMap.put(family, kvs);
783       region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
784     } catch (Exception e) {
785       ok = true;
786     }
787     assertEquals("Family " +new String(family)+ " does exist", true, ok);
788   }
789 
790   public void testDelete_mixed() throws IOException, InterruptedException {
791     byte [] tableName = Bytes.toBytes("testtable");
792     byte [] fam = Bytes.toBytes("info");
793     byte [][] families = {fam};
794     String method = this.getName();
795     initHRegion(tableName, method, families);
796     EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
797 
798     byte [] row = Bytes.toBytes("table_name");
799     // column names
800     byte [] serverinfo = Bytes.toBytes("serverinfo");
801     byte [] splitA = Bytes.toBytes("splitA");
802     byte [] splitB = Bytes.toBytes("splitB");
803 
804     // add some data:
805     Put put = new Put(row);
806     put.add(fam, splitA, Bytes.toBytes("reference_A"));
807     region.put(put);
808 
809     put = new Put(row);
810     put.add(fam, splitB, Bytes.toBytes("reference_B"));
811     region.put(put);
812 
813     put = new Put(row);
814     put.add(fam, serverinfo, Bytes.toBytes("ip_address"));
815     region.put(put);
816 
817     // ok now delete a split:
818     Delete delete = new Delete(row);
819     delete.deleteColumns(fam, splitA);
820     region.delete(delete, null, true);
821 
822     // assert some things:
823     Get get = new Get(row).addColumn(fam, serverinfo);
824     Result result = region.get(get, null);
825     assertEquals(1, result.size());
826 
827     get = new Get(row).addColumn(fam, splitA);
828     result = region.get(get, null);
829     assertEquals(0, result.size());
830 
831     get = new Get(row).addColumn(fam, splitB);
832     result = region.get(get, null);
833     assertEquals(1, result.size());
834 
835     // Assert that after a delete, I can put.
836     put = new Put(row);
837     put.add(fam, splitA, Bytes.toBytes("reference_A"));
838     region.put(put);
839     get = new Get(row);
840     result = region.get(get, null);
841     assertEquals(3, result.size());
842 
843     // Now delete all... then test I can add stuff back
844     delete = new Delete(row);
845     region.delete(delete, null, false);
846     assertEquals(0, region.get(get, null).size());
847 
848     region.put(new Put(row).add(fam, splitA, Bytes.toBytes("reference_A")));
849     result = region.get(get, null);
850     assertEquals(1, result.size());
851   }
852 
853   public void testDeleteRowWithFutureTs() throws IOException {
854     byte [] tableName = Bytes.toBytes("testtable");
855     byte [] fam = Bytes.toBytes("info");
856     byte [][] families = {fam};
857     String method = this.getName();
858     initHRegion(tableName, method, families);
859 
860     byte [] row = Bytes.toBytes("table_name");
861     // column names
862     byte [] serverinfo = Bytes.toBytes("serverinfo");
863 
864     // add data in the far future
865     Put put = new Put(row);
866     put.add(fam, serverinfo, HConstants.LATEST_TIMESTAMP-5,Bytes.toBytes("value"));
867     region.put(put);
868 
869     // now delete something in the present
870     Delete delete = new Delete(row);
871     region.delete(delete, null, true);
872 
873     // make sure we still see our data
874     Get get = new Get(row).addColumn(fam, serverinfo);
875     Result result = region.get(get, null);
876     assertEquals(1, result.size());
877 
878     // delete the future row
879     delete = new Delete(row,HConstants.LATEST_TIMESTAMP-3,null);
880     region.delete(delete, null, true);
881 
882     // make sure it is gone
883     get = new Get(row).addColumn(fam, serverinfo);
884     result = region.get(get, null);
885     assertEquals(0, result.size());
886   }
887 
888   /**
889    * Tests that the special LATEST_TIMESTAMP option for puts gets
890    * replaced by the actual timestamp
891    */
892   public void testPutWithLatestTS() throws IOException {
893     byte [] tableName = Bytes.toBytes("testtable");
894     byte [] fam = Bytes.toBytes("info");
895     byte [][] families = {fam};
896     String method = this.getName();
897     initHRegion(tableName, method, families);
898 
899     byte [] row = Bytes.toBytes("row1");
900     // column names
901     byte [] qual = Bytes.toBytes("qual");
902 
903     // add data with LATEST_TIMESTAMP, put without WAL
904     Put put = new Put(row);
905     put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
906     region.put(put, false);
907 
908     // Make sure it shows up with an actual timestamp
909     Get get = new Get(row).addColumn(fam, qual);
910     Result result = region.get(get, null);
911     assertEquals(1, result.size());
912     KeyValue kv = result.raw()[0];
913     LOG.info("Got: " + kv);
914     assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
915         kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
916 
917     // Check same with WAL enabled (historically these took different
918     // code paths, so check both)
919     row = Bytes.toBytes("row2");
920     put = new Put(row);
921     put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
922     region.put(put, true);
923 
924     // Make sure it shows up with an actual timestamp
925     get = new Get(row).addColumn(fam, qual);
926     result = region.get(get, null);
927     assertEquals(1, result.size());
928     kv = result.raw()[0];
929     LOG.info("Got: " + kv);
930     assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
931         kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
932 
933   }
934 
935   public void testScanner_DeleteOneFamilyNotAnother() throws IOException {
936     byte [] tableName = Bytes.toBytes("test_table");
937     byte [] fam1 = Bytes.toBytes("columnA");
938     byte [] fam2 = Bytes.toBytes("columnB");
939     initHRegion(tableName, getName(), fam1, fam2);
940 
941     byte [] rowA = Bytes.toBytes("rowA");
942     byte [] rowB = Bytes.toBytes("rowB");
943 
944     byte [] value = Bytes.toBytes("value");
945 
946     Delete delete = new Delete(rowA);
947     delete.deleteFamily(fam1);
948 
949     region.delete(delete, null, true);
950 
951     // now create data.
952     Put put = new Put(rowA);
953     put.add(fam2, null, value);
954     region.put(put);
955 
956     put = new Put(rowB);
957     put.add(fam1, null, value);
958     put.add(fam2, null, value);
959     region.put(put);
960 
961     Scan scan = new Scan();
962     scan.addFamily(fam1).addFamily(fam2);
963     InternalScanner s = region.getScanner(scan);
964     List<KeyValue> results = new ArrayList<KeyValue>();
965     s.next(results);
966     assertTrue(Bytes.equals(rowA, results.get(0).getRow()));
967 
968     results.clear();
969     s.next(results);
970     assertTrue(Bytes.equals(rowB, results.get(0).getRow()));
971 
972   }
973 
974   public void testDeleteColumns_PostInsert() throws IOException,
975       InterruptedException {
976     Delete delete = new Delete(row);
977     delete.deleteColumns(fam1, qual1);
978     doTestDelete_AndPostInsert(delete);
979   }
980 
981   public void testDeleteFamily_PostInsert() throws IOException, InterruptedException {
982     Delete delete = new Delete(row);
983     delete.deleteFamily(fam1);
984     doTestDelete_AndPostInsert(delete);
985   }
986 
987   public void doTestDelete_AndPostInsert(Delete delete)
988       throws IOException, InterruptedException {
989     initHRegion(tableName, getName(), fam1);
990     EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
991     Put put = new Put(row);
992     put.add(fam1, qual1, value1);
993     region.put(put);
994 
995     // now delete the value:
996     region.delete(delete, null, true);
997 
998 
999     // ok put data:
1000     put = new Put(row);
1001     put.add(fam1, qual1, value2);
1002     region.put(put);
1003 
1004     // ok get:
1005     Get get = new Get(row);
1006     get.addColumn(fam1, qual1);
1007 
1008     Result r = region.get(get, null);
1009     assertEquals(1, r.size());
1010     assertByteEquals(value2, r.getValue(fam1, qual1));
1011 
1012     // next:
1013     Scan scan = new Scan(row);
1014     scan.addColumn(fam1, qual1);
1015     InternalScanner s = region.getScanner(scan);
1016 
1017     List<KeyValue> results = new ArrayList<KeyValue>();
1018     assertEquals(false, s.next(results));
1019     assertEquals(1, results.size());
1020     KeyValue kv = results.get(0);
1021 
1022     assertByteEquals(value2, kv.getValue());
1023     assertByteEquals(fam1, kv.getFamily());
1024     assertByteEquals(qual1, kv.getQualifier());
1025     assertByteEquals(row, kv.getRow());
1026   }
1027 
1028 
1029 
1030   public void testDelete_CheckTimestampUpdated()
1031   throws IOException {
1032     byte [] row1 = Bytes.toBytes("row1");
1033     byte [] col1 = Bytes.toBytes("col1");
1034     byte [] col2 = Bytes.toBytes("col2");
1035     byte [] col3 = Bytes.toBytes("col3");
1036 
1037     //Setting up region
1038     String method = this.getName();
1039     initHRegion(tableName, method, fam1);
1040 
1041     //Building checkerList
1042     List<KeyValue> kvs  = new ArrayList<KeyValue>();
1043     kvs.add(new KeyValue(row1, fam1, col1, null));
1044     kvs.add(new KeyValue(row1, fam1, col2, null));
1045     kvs.add(new KeyValue(row1, fam1, col3, null));
1046 
1047     Map<byte[], List<KeyValue>> deleteMap = new HashMap<byte[], List<KeyValue>>();
1048     deleteMap.put(fam1, kvs);
1049     region.delete(deleteMap, HConstants.DEFAULT_CLUSTER_ID, true);
1050 
1051     // extract the key values out the memstore:
1052     // This is kinda hacky, but better than nothing...
1053     long now = System.currentTimeMillis();
1054     KeyValue firstKv = region.getStore(fam1).memstore.kvset.first();
1055     assertTrue(firstKv.getTimestamp() <= now);
1056     now = firstKv.getTimestamp();
1057     for (KeyValue kv: region.getStore(fam1).memstore.kvset) {
1058       assertTrue(kv.getTimestamp() <= now);
1059       now = kv.getTimestamp();
1060     }
1061   }
1062 
1063   //////////////////////////////////////////////////////////////////////////////
1064   // Get tests
1065   //////////////////////////////////////////////////////////////////////////////
1066   public void testGet_FamilyChecker() throws IOException {
1067     byte [] tableName = Bytes.toBytes("testtable");
1068     byte [] row1 = Bytes.toBytes("row1");
1069     byte [] fam1 = Bytes.toBytes("fam1");
1070     byte [] fam2 = Bytes.toBytes("False");
1071     byte [] col1 = Bytes.toBytes("col1");
1072 
1073     //Setting up region
1074     String method = this.getName();
1075     initHRegion(tableName, method, fam1);
1076 
1077     Get get = new Get(row1);
1078     get.addColumn(fam2, col1);
1079 
1080     //Test
1081     try {
1082       region.get(get, null);
1083     } catch (NoSuchColumnFamilyException e){
1084       assertFalse(false);
1085       return;
1086     }
1087     assertFalse(true);
1088   }
1089 
1090   public void testGet_Basic() throws IOException {
1091     byte [] tableName = Bytes.toBytes("testtable");
1092     byte [] row1 = Bytes.toBytes("row1");
1093     byte [] fam1 = Bytes.toBytes("fam1");
1094     byte [] col1 = Bytes.toBytes("col1");
1095     byte [] col2 = Bytes.toBytes("col2");
1096     byte [] col3 = Bytes.toBytes("col3");
1097     byte [] col4 = Bytes.toBytes("col4");
1098     byte [] col5 = Bytes.toBytes("col5");
1099 
1100     //Setting up region
1101     String method = this.getName();
1102     initHRegion(tableName, method, fam1);
1103 
1104     //Add to memstore
1105     Put put = new Put(row1);
1106     put.add(fam1, col1, null);
1107     put.add(fam1, col2, null);
1108     put.add(fam1, col3, null);
1109     put.add(fam1, col4, null);
1110     put.add(fam1, col5, null);
1111     region.put(put);
1112 
1113     Get get = new Get(row1);
1114     get.addColumn(fam1, col2);
1115     get.addColumn(fam1, col4);
1116     //Expected result
1117     KeyValue kv1 = new KeyValue(row1, fam1, col2);
1118     KeyValue kv2 = new KeyValue(row1, fam1, col4);
1119     KeyValue [] expected = {kv1, kv2};
1120 
1121     //Test
1122     Result res = region.get(get, null);
1123     assertEquals(expected.length, res.size());
1124     for(int i=0; i<res.size(); i++){
1125       assertEquals(0,
1126           Bytes.compareTo(expected[i].getRow(), res.raw()[i].getRow()));
1127       assertEquals(0,
1128           Bytes.compareTo(expected[i].getFamily(), res.raw()[i].getFamily()));
1129       assertEquals(0,
1130           Bytes.compareTo(
1131               expected[i].getQualifier(), res.raw()[i].getQualifier()));
1132     }
1133 
1134     // Test using a filter on a Get
1135     Get g = new Get(row1);
1136     final int count = 2;
1137     g.setFilter(new ColumnCountGetFilter(count));
1138     res = region.get(g, null);
1139     assertEquals(count, res.size());
1140   }
1141 
1142   public void testGet_Empty() throws IOException {
1143     byte [] tableName = Bytes.toBytes("emptytable");
1144     byte [] row = Bytes.toBytes("row");
1145     byte [] fam = Bytes.toBytes("fam");
1146 
1147     String method = this.getName();
1148     initHRegion(tableName, method, fam);
1149 
1150     Get get = new Get(row);
1151     get.addFamily(fam);
1152     Result r = region.get(get, null);
1153 
1154     assertTrue(r.isEmpty());
1155   }
1156 
1157   //Test that checked if there was anything special when reading from the ROOT
1158   //table. To be able to use this test you need to comment the part in
1159   //HTableDescriptor that checks for '-' and '.'. You also need to remove the
1160   //s in the beginning of the name.
1161   public void stestGet_Root() throws IOException {
1162     //Setting up region
1163     String method = this.getName();
1164     initHRegion(HConstants.ROOT_TABLE_NAME, method, HConstants.CATALOG_FAMILY);
1165 
1166     //Add to memstore
1167     Put put = new Put(HConstants.EMPTY_START_ROW);
1168     put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER, null);
1169     region.put(put);
1170 
1171     Get get = new Get(HConstants.EMPTY_START_ROW);
1172     get.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
1173 
1174     //Expected result
1175     KeyValue kv1 = new KeyValue(HConstants.EMPTY_START_ROW,
1176         HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
1177     KeyValue [] expected = {kv1};
1178 
1179     //Test from memstore
1180     Result res = region.get(get, null);
1181 
1182     assertEquals(expected.length, res.size());
1183     for(int i=0; i<res.size(); i++){
1184       assertEquals(0,
1185           Bytes.compareTo(expected[i].getRow(), res.raw()[i].getRow()));
1186       assertEquals(0,
1187           Bytes.compareTo(expected[i].getFamily(), res.raw()[i].getFamily()));
1188       assertEquals(0,
1189           Bytes.compareTo(
1190               expected[i].getQualifier(), res.raw()[i].getQualifier()));
1191     }
1192 
1193     //flush
1194     region.flushcache();
1195 
1196     //test2
1197     res = region.get(get, null);
1198 
1199     assertEquals(expected.length, res.size());
1200     for(int i=0; i<res.size(); i++){
1201       assertEquals(0,
1202           Bytes.compareTo(expected[i].getRow(), res.raw()[i].getRow()));
1203       assertEquals(0,
1204           Bytes.compareTo(expected[i].getFamily(), res.raw()[i].getFamily()));
1205       assertEquals(0,
1206           Bytes.compareTo(
1207               expected[i].getQualifier(), res.raw()[i].getQualifier()));
1208     }
1209 
1210     //Scan
1211     Scan scan = new Scan();
1212     scan.addFamily(HConstants.CATALOG_FAMILY);
1213     InternalScanner s = region.getScanner(scan);
1214     List<KeyValue> result = new ArrayList<KeyValue>();
1215     s.next(result);
1216 
1217     assertEquals(expected.length, result.size());
1218     for(int i=0; i<res.size(); i++){
1219       assertEquals(0,
1220           Bytes.compareTo(expected[i].getRow(), result.get(i).getRow()));
1221       assertEquals(0,
1222           Bytes.compareTo(expected[i].getFamily(), result.get(i).getFamily()));
1223       assertEquals(0,
1224           Bytes.compareTo(
1225               expected[i].getQualifier(), result.get(i).getQualifier()));
1226     }
1227   }
1228 
1229   //////////////////////////////////////////////////////////////////////////////
1230   // Lock test
1231   //////////////////////////////////////////////////////////////////////////////
1232   public void testLocks() throws IOException{
1233     byte [] tableName = Bytes.toBytes("testtable");
1234     byte [][] families = {fam1, fam2, fam3};
1235 
1236     Configuration hc = initSplit();
1237     //Setting up region
1238     String method = this.getName();
1239     initHRegion(tableName, method, hc, families);
1240 
1241     final int threadCount = 10;
1242     final int lockCount = 10;
1243 
1244     List<Thread>threads = new ArrayList<Thread>(threadCount);
1245     for (int i = 0; i < threadCount; i++) {
1246       threads.add(new Thread(Integer.toString(i)) {
1247         @Override
1248         public void run() {
1249           Integer [] lockids = new Integer[lockCount];
1250           // Get locks.
1251           for (int i = 0; i < lockCount; i++) {
1252             try {
1253               byte [] rowid = Bytes.toBytes(Integer.toString(i));
1254               lockids[i] = region.obtainRowLock(rowid);
1255               assertEquals(rowid, region.getRowFromLock(lockids[i]));
1256               LOG.debug(getName() + " locked " + Bytes.toString(rowid));
1257             } catch (IOException e) {
1258               e.printStackTrace();
1259             }
1260           }
1261           LOG.debug(getName() + " set " +
1262               Integer.toString(lockCount) + " locks");
1263 
1264           // Abort outstanding locks.
1265           for (int i = lockCount - 1; i >= 0; i--) {
1266             region.releaseRowLock(lockids[i]);
1267             LOG.debug(getName() + " unlocked " + i);
1268           }
1269           LOG.debug(getName() + " released " +
1270               Integer.toString(lockCount) + " locks");
1271         }
1272       });
1273     }
1274 
1275     // Startup all our threads.
1276     for (Thread t : threads) {
1277       t.start();
1278     }
1279 
1280     // Now wait around till all are done.
1281     for (Thread t: threads) {
1282       while (t.isAlive()) {
1283         try {
1284           Thread.sleep(1);
1285         } catch (InterruptedException e) {
1286           // Go around again.
1287         }
1288       }
1289     }
1290     LOG.info("locks completed.");
1291   }
1292 
1293   //////////////////////////////////////////////////////////////////////////////
1294   // Merge test
1295   //////////////////////////////////////////////////////////////////////////////
1296   public void testMerge() throws IOException {
1297     byte [] tableName = Bytes.toBytes("testtable");
1298     byte [][] families = {fam1, fam2, fam3};
1299     Configuration hc = initSplit();
1300     //Setting up region
1301     String method = this.getName();
1302     initHRegion(tableName, method, hc, families);
1303     try {
1304       LOG.info("" + addContent(region, fam3));
1305       region.flushcache();
1306       region.compactStores();
1307       byte [] splitRow = region.checkSplit();
1308       assertNotNull(splitRow);
1309       LOG.info("SplitRow: " + Bytes.toString(splitRow));
1310       HRegion [] subregions = splitRegion(region, splitRow);
1311       try {
1312         // Need to open the regions.
1313         for (int i = 0; i < subregions.length; i++) {
1314           openClosedRegion(subregions[i]);
1315           subregions[i].compactStores();
1316         }
1317         Path oldRegionPath = region.getRegionDir();
1318         Path oldRegion1 = subregions[0].getRegionDir();
1319         Path oldRegion2 = subregions[1].getRegionDir();
1320         long startTime = System.currentTimeMillis();
1321         region = HRegion.mergeAdjacent(subregions[0], subregions[1]);
1322         LOG.info("Merge regions elapsed time: " +
1323             ((System.currentTimeMillis() - startTime) / 1000.0));
1324         fs.delete(oldRegion1, true);
1325         fs.delete(oldRegion2, true);
1326         fs.delete(oldRegionPath, true);
1327         LOG.info("splitAndMerge completed.");
1328       } finally {
1329         for (int i = 0; i < subregions.length; i++) {
1330           try {
1331             subregions[i].close();
1332           } catch (IOException e) {
1333             // Ignore.
1334           }
1335         }
1336       }
1337     } finally {
1338       if (region != null) {
1339         region.close();
1340         region.getLog().closeAndDelete();
1341       }
1342     }
1343   }
1344 
1345   /**
1346    * @param parent Region to split.
1347    * @param midkey Key to split around.
1348    * @return The Regions we created.
1349    * @throws IOException
1350    */
1351   HRegion [] splitRegion(final HRegion parent, final byte [] midkey)
1352   throws IOException {
1353     PairOfSameType<HRegion> result = null;
1354     SplitTransaction st = new SplitTransaction(parent, midkey);
1355     // If prepare does not return true, for some reason -- logged inside in
1356     // the prepare call -- we are not ready to split just now.  Just return.
1357     if (!st.prepare()) return null;
1358     try {
1359       result = st.execute(null, null);
1360     } catch (IOException ioe) {
1361       try {
1362         LOG.info("Running rollback of failed split of " +
1363           parent.getRegionNameAsString() + "; " + ioe.getMessage());
1364         st.rollback(null, null);
1365         LOG.info("Successful rollback of failed split of " +
1366           parent.getRegionNameAsString());
1367         return null;
1368       } catch (RuntimeException e) {
1369         // If failed rollback, kill this server to avoid having a hole in table.
1370         LOG.info("Failed rollback of failed split of " +
1371           parent.getRegionNameAsString() + " -- aborting server", e);
1372       }
1373     }
1374     return new HRegion [] {result.getFirst(), result.getSecond()};
1375   }
1376 
1377   //////////////////////////////////////////////////////////////////////////////
1378   // Scanner tests
1379   //////////////////////////////////////////////////////////////////////////////
1380   public void testGetScanner_WithOkFamilies() throws IOException {
1381     byte [] tableName = Bytes.toBytes("testtable");
1382     byte [] fam1 = Bytes.toBytes("fam1");
1383     byte [] fam2 = Bytes.toBytes("fam2");
1384 
1385     byte [][] families = {fam1, fam2};
1386 
1387     //Setting up region
1388     String method = this.getName();
1389     initHRegion(tableName, method, families);
1390 
1391     Scan scan = new Scan();
1392     scan.addFamily(fam1);
1393     scan.addFamily(fam2);
1394     try {
1395       region.getScanner(scan);
1396     } catch (Exception e) {
1397       assertTrue("Families could not be found in Region", false);
1398     }
1399   }
1400 
1401   public void testGetScanner_WithNotOkFamilies() throws IOException {
1402     byte [] tableName = Bytes.toBytes("testtable");
1403     byte [] fam1 = Bytes.toBytes("fam1");
1404     byte [] fam2 = Bytes.toBytes("fam2");
1405 
1406     byte [][] families = {fam1};
1407 
1408     //Setting up region
1409     String method = this.getName();
1410     initHRegion(tableName, method, families);
1411 
1412     Scan scan = new Scan();
1413     scan.addFamily(fam2);
1414     boolean ok = false;
1415     try {
1416       region.getScanner(scan);
1417     } catch (Exception e) {
1418       ok = true;
1419     }
1420     assertTrue("Families could not be found in Region", ok);
1421   }
1422 
1423   public void testGetScanner_WithNoFamilies() throws IOException {
1424     byte [] tableName = Bytes.toBytes("testtable");
1425     byte [] row1 = Bytes.toBytes("row1");
1426     byte [] fam1 = Bytes.toBytes("fam1");
1427     byte [] fam2 = Bytes.toBytes("fam2");
1428     byte [] fam3 = Bytes.toBytes("fam3");
1429     byte [] fam4 = Bytes.toBytes("fam4");
1430 
1431     byte [][] families = {fam1, fam2, fam3, fam4};
1432 
1433     //Setting up region
1434     String method = this.getName();
1435     initHRegion(tableName, method, families);
1436 
1437 
1438     //Putting data in Region
1439     Put put = new Put(row1);
1440     put.add(fam1, null, null);
1441     put.add(fam2, null, null);
1442     put.add(fam3, null, null);
1443     put.add(fam4, null, null);
1444     region.put(put);
1445 
1446     Scan scan = null;
1447     HRegion.RegionScannerImpl is = null;
1448 
1449     //Testing to see how many scanners that is produced by getScanner, starting
1450     //with known number, 2 - current = 1
1451     scan = new Scan();
1452     scan.addFamily(fam2);
1453     scan.addFamily(fam4);
1454     is = (RegionScannerImpl) region.getScanner(scan);
1455     MultiVersionConsistencyControl.resetThreadReadPoint(region.getMVCC());
1456     assertEquals(1, ((RegionScannerImpl)is).storeHeap.getHeap().size());
1457 
1458     scan = new Scan();
1459     is = (RegionScannerImpl) region.getScanner(scan);
1460     MultiVersionConsistencyControl.resetThreadReadPoint(region.getMVCC());
1461     assertEquals(families.length -1,
1462         ((RegionScannerImpl)is).storeHeap.getHeap().size());
1463   }
1464 
1465   /**
1466    * This method tests https://issues.apache.org/jira/browse/HBASE-2516.
1467    */
1468   public void testGetScanner_WithRegionClosed() {
1469     byte[] tableName = Bytes.toBytes("testtable");
1470     byte[] fam1 = Bytes.toBytes("fam1");
1471     byte[] fam2 = Bytes.toBytes("fam2");
1472 
1473     byte[][] families = {fam1, fam2};
1474 
1475     //Setting up region
1476     String method = this.getName();
1477     try {
1478       initHRegion(tableName, method, families);
1479     } catch (IOException e) {
1480       e.printStackTrace();
1481       fail("Got IOException during initHRegion, " + e.getMessage());
1482     }
1483     region.closed.set(true);
1484     try {
1485       region.getScanner(null);
1486       fail("Expected to get an exception during getScanner on a region that is closed");
1487     } catch (org.apache.hadoop.hbase.NotServingRegionException e) {
1488       //this is the correct exception that is expected
1489     } catch (IOException e) {
1490       fail("Got wrong type of exception - should be a NotServingRegionException, but was an IOException: "
1491               + e.getMessage());
1492     }
1493   }
1494 
1495   public void testRegionScanner_Next() throws IOException {
1496     byte [] tableName = Bytes.toBytes("testtable");
1497     byte [] row1 = Bytes.toBytes("row1");
1498     byte [] row2 = Bytes.toBytes("row2");
1499     byte [] fam1 = Bytes.toBytes("fam1");
1500     byte [] fam2 = Bytes.toBytes("fam2");
1501     byte [] fam3 = Bytes.toBytes("fam3");
1502     byte [] fam4 = Bytes.toBytes("fam4");
1503 
1504     byte [][] families = {fam1, fam2, fam3, fam4};
1505     long ts = System.currentTimeMillis();
1506 
1507     //Setting up region
1508     String method = this.getName();
1509     initHRegion(tableName, method, families);
1510 
1511     //Putting data in Region
1512     Put put = null;
1513     put = new Put(row1);
1514     put.add(fam1, null, ts, null);
1515     put.add(fam2, null, ts, null);
1516     put.add(fam3, null, ts, null);
1517     put.add(fam4, null, ts, null);
1518     region.put(put);
1519 
1520     put = new Put(row2);
1521     put.add(fam1, null, ts, null);
1522     put.add(fam2, null, ts, null);
1523     put.add(fam3, null, ts, null);
1524     put.add(fam4, null, ts, null);
1525     region.put(put);
1526 
1527     Scan scan = new Scan();
1528     scan.addFamily(fam2);
1529     scan.addFamily(fam4);
1530     InternalScanner is = region.getScanner(scan);
1531 
1532     List<KeyValue> res = null;
1533 
1534     //Result 1
1535     List<KeyValue> expected1 = new ArrayList<KeyValue>();
1536     expected1.add(new KeyValue(row1, fam2, null, ts, KeyValue.Type.Put, null));
1537     expected1.add(new KeyValue(row1, fam4, null, ts, KeyValue.Type.Put, null));
1538 
1539     res = new ArrayList<KeyValue>();
1540     is.next(res);
1541     for(int i=0; i<res.size(); i++) {
1542       assertEquals(expected1.get(i), res.get(i));
1543     }
1544 
1545     //Result 2
1546     List<KeyValue> expected2 = new ArrayList<KeyValue>();
1547     expected2.add(new KeyValue(row2, fam2, null, ts, KeyValue.Type.Put, null));
1548     expected2.add(new KeyValue(row2, fam4, null, ts, KeyValue.Type.Put, null));
1549 
1550     res = new ArrayList<KeyValue>();
1551     is.next(res);
1552     for(int i=0; i<res.size(); i++) {
1553       assertEquals(expected2.get(i), res.get(i));
1554     }
1555 
1556   }
1557 
1558   public void testScanner_ExplicitColumns_FromMemStore_EnforceVersions()
1559   throws IOException {
1560     byte [] tableName = Bytes.toBytes("testtable");
1561     byte [] row1 = Bytes.toBytes("row1");
1562     byte [] qf1 = Bytes.toBytes("qualifier1");
1563     byte [] qf2 = Bytes.toBytes("qualifier2");
1564     byte [] fam1 = Bytes.toBytes("fam1");
1565     byte [][] families = {fam1};
1566 
1567     long ts1 = System.currentTimeMillis();
1568     long ts2 = ts1 + 1;
1569     long ts3 = ts1 + 2;
1570 
1571     //Setting up region
1572     String method = this.getName();
1573     initHRegion(tableName, method, families);
1574 
1575     //Putting data in Region
1576     Put put = null;
1577     KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
1578     KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
1579     KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
1580 
1581     KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
1582     KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
1583     KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
1584 
1585     put = new Put(row1);
1586     put.add(kv13);
1587     put.add(kv12);
1588     put.add(kv11);
1589     put.add(kv23);
1590     put.add(kv22);
1591     put.add(kv21);
1592     region.put(put);
1593 
1594     //Expected
1595     List<KeyValue> expected = new ArrayList<KeyValue>();
1596     expected.add(kv13);
1597     expected.add(kv12);
1598 
1599     Scan scan = new Scan(row1);
1600     scan.addColumn(fam1, qf1);
1601     scan.setMaxVersions(MAX_VERSIONS);
1602     List<KeyValue> actual = new ArrayList<KeyValue>();
1603     InternalScanner scanner = region.getScanner(scan);
1604 
1605     boolean hasNext = scanner.next(actual);
1606     assertEquals(false, hasNext);
1607 
1608     //Verify result
1609     for(int i=0; i<expected.size(); i++) {
1610       assertEquals(expected.get(i), actual.get(i));
1611     }
1612   }
1613 
1614   public void testScanner_ExplicitColumns_FromFilesOnly_EnforceVersions()
1615   throws IOException{
1616     byte [] tableName = Bytes.toBytes("testtable");
1617     byte [] row1 = Bytes.toBytes("row1");
1618     byte [] qf1 = Bytes.toBytes("qualifier1");
1619     byte [] qf2 = Bytes.toBytes("qualifier2");
1620     byte [] fam1 = Bytes.toBytes("fam1");
1621     byte [][] families = {fam1};
1622 
1623     long ts1 = 1; //System.currentTimeMillis();
1624     long ts2 = ts1 + 1;
1625     long ts3 = ts1 + 2;
1626 
1627     //Setting up region
1628     String method = this.getName();
1629     initHRegion(tableName, method, families);
1630 
1631     //Putting data in Region
1632     Put put = null;
1633     KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
1634     KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
1635     KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
1636 
1637     KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
1638     KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
1639     KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
1640 
1641     put = new Put(row1);
1642     put.add(kv13);
1643     put.add(kv12);
1644     put.add(kv11);
1645     put.add(kv23);
1646     put.add(kv22);
1647     put.add(kv21);
1648     region.put(put);
1649     region.flushcache();
1650 
1651     //Expected
1652     List<KeyValue> expected = new ArrayList<KeyValue>();
1653     expected.add(kv13);
1654     expected.add(kv12);
1655     expected.add(kv23);
1656     expected.add(kv22);
1657 
1658     Scan scan = new Scan(row1);
1659     scan.addColumn(fam1, qf1);
1660     scan.addColumn(fam1, qf2);
1661     scan.setMaxVersions(MAX_VERSIONS);
1662     List<KeyValue> actual = new ArrayList<KeyValue>();
1663     InternalScanner scanner = region.getScanner(scan);
1664 
1665     boolean hasNext = scanner.next(actual);
1666     assertEquals(false, hasNext);
1667 
1668     //Verify result
1669     for(int i=0; i<expected.size(); i++) {
1670       assertEquals(expected.get(i), actual.get(i));
1671     }
1672   }
1673 
1674   public void testScanner_ExplicitColumns_FromMemStoreAndFiles_EnforceVersions()
1675   throws IOException {
1676     byte [] tableName = Bytes.toBytes("testtable");
1677     byte [] row1 = Bytes.toBytes("row1");
1678     byte [] fam1 = Bytes.toBytes("fam1");
1679     byte [][] families = {fam1};
1680     byte [] qf1 = Bytes.toBytes("qualifier1");
1681     byte [] qf2 = Bytes.toBytes("qualifier2");
1682 
1683     long ts1 = 1;
1684     long ts2 = ts1 + 1;
1685     long ts3 = ts1 + 2;
1686     long ts4 = ts1 + 3;
1687 
1688     //Setting up region
1689     String method = this.getName();
1690     initHRegion(tableName, method, families);
1691 
1692     //Putting data in Region
1693     KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
1694     KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
1695     KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
1696     KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
1697 
1698     KeyValue kv24 = new KeyValue(row1, fam1, qf2, ts4, KeyValue.Type.Put, null);
1699     KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
1700     KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
1701     KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
1702 
1703     Put put = null;
1704     put = new Put(row1);
1705     put.add(kv14);
1706     put.add(kv24);
1707     region.put(put);
1708     region.flushcache();
1709 
1710     put = new Put(row1);
1711     put.add(kv23);
1712     put.add(kv13);
1713     region.put(put);
1714     region.flushcache();
1715 
1716     put = new Put(row1);
1717     put.add(kv22);
1718     put.add(kv12);
1719     region.put(put);
1720     region.flushcache();
1721 
1722     put = new Put(row1);
1723     put.add(kv21);
1724     put.add(kv11);
1725     region.put(put);
1726 
1727     //Expected
1728     List<KeyValue> expected = new ArrayList<KeyValue>();
1729     expected.add(kv14);
1730     expected.add(kv13);
1731     expected.add(kv12);
1732     expected.add(kv24);
1733     expected.add(kv23);
1734     expected.add(kv22);
1735 
1736     Scan scan = new Scan(row1);
1737     scan.addColumn(fam1, qf1);
1738     scan.addColumn(fam1, qf2);
1739     int versions = 3;
1740     scan.setMaxVersions(versions);
1741     List<KeyValue> actual = new ArrayList<KeyValue>();
1742     InternalScanner scanner = region.getScanner(scan);
1743 
1744     boolean hasNext = scanner.next(actual);
1745     assertEquals(false, hasNext);
1746 
1747     //Verify result
1748     for(int i=0; i<expected.size(); i++) {
1749       assertEquals(expected.get(i), actual.get(i));
1750     }
1751   }
1752 
1753   public void testScanner_Wildcard_FromMemStore_EnforceVersions()
1754   throws IOException {
1755     byte [] tableName = Bytes.toBytes("testtable");
1756     byte [] row1 = Bytes.toBytes("row1");
1757     byte [] qf1 = Bytes.toBytes("qualifier1");
1758     byte [] qf2 = Bytes.toBytes("qualifier2");
1759     byte [] fam1 = Bytes.toBytes("fam1");
1760     byte [][] families = {fam1};
1761 
1762     long ts1 = System.currentTimeMillis();
1763     long ts2 = ts1 + 1;
1764     long ts3 = ts1 + 2;
1765 
1766     //Setting up region
1767     String method = this.getName();
1768     initHRegion(tableName, method, families);
1769 
1770     //Putting data in Region
1771     Put put = null;
1772     KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
1773     KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
1774     KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
1775 
1776     KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
1777     KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
1778     KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
1779 
1780     put = new Put(row1);
1781     put.add(kv13);
1782     put.add(kv12);
1783     put.add(kv11);
1784     put.add(kv23);
1785     put.add(kv22);
1786     put.add(kv21);
1787     region.put(put);
1788 
1789     //Expected
1790     List<KeyValue> expected = new ArrayList<KeyValue>();
1791     expected.add(kv13);
1792     expected.add(kv12);
1793     expected.add(kv23);
1794     expected.add(kv22);
1795 
1796     Scan scan = new Scan(row1);
1797     scan.addFamily(fam1);
1798     scan.setMaxVersions(MAX_VERSIONS);
1799     List<KeyValue> actual = new ArrayList<KeyValue>();
1800     InternalScanner scanner = region.getScanner(scan);
1801 
1802     boolean hasNext = scanner.next(actual);
1803     assertEquals(false, hasNext);
1804 
1805     //Verify result
1806     for(int i=0; i<expected.size(); i++) {
1807       assertEquals(expected.get(i), actual.get(i));
1808     }
1809   }
1810 
1811   public void testScanner_Wildcard_FromFilesOnly_EnforceVersions()
1812   throws IOException{
1813     byte [] tableName = Bytes.toBytes("testtable");
1814     byte [] row1 = Bytes.toBytes("row1");
1815     byte [] qf1 = Bytes.toBytes("qualifier1");
1816     byte [] qf2 = Bytes.toBytes("qualifier2");
1817     byte [] fam1 = Bytes.toBytes("fam1");
1818 
1819     long ts1 = 1; //System.currentTimeMillis();
1820     long ts2 = ts1 + 1;
1821     long ts3 = ts1 + 2;
1822 
1823     //Setting up region
1824     String method = this.getName();
1825     initHRegion(tableName, method, fam1);
1826 
1827     //Putting data in Region
1828     Put put = null;
1829     KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
1830     KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
1831     KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
1832 
1833     KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
1834     KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
1835     KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
1836 
1837     put = new Put(row1);
1838     put.add(kv13);
1839     put.add(kv12);
1840     put.add(kv11);
1841     put.add(kv23);
1842     put.add(kv22);
1843     put.add(kv21);
1844     region.put(put);
1845     region.flushcache();
1846 
1847     //Expected
1848     List<KeyValue> expected = new ArrayList<KeyValue>();
1849     expected.add(kv13);
1850     expected.add(kv12);
1851     expected.add(kv23);
1852     expected.add(kv22);
1853 
1854     Scan scan = new Scan(row1);
1855     scan.addFamily(fam1);
1856     scan.setMaxVersions(MAX_VERSIONS);
1857     List<KeyValue> actual = new ArrayList<KeyValue>();
1858     InternalScanner scanner = region.getScanner(scan);
1859 
1860     boolean hasNext = scanner.next(actual);
1861     assertEquals(false, hasNext);
1862 
1863     //Verify result
1864     for(int i=0; i<expected.size(); i++) {
1865       assertEquals(expected.get(i), actual.get(i));
1866     }
1867   }
1868 
1869   public void testScanner_StopRow1542() throws IOException {
1870     byte [] tableName = Bytes.toBytes("test_table");
1871     byte [] family = Bytes.toBytes("testFamily");
1872     initHRegion(tableName, getName(), family);
1873 
1874     byte [] row1 = Bytes.toBytes("row111");
1875     byte [] row2 = Bytes.toBytes("row222");
1876     byte [] row3 = Bytes.toBytes("row333");
1877     byte [] row4 = Bytes.toBytes("row444");
1878     byte [] row5 = Bytes.toBytes("row555");
1879 
1880     byte [] col1 = Bytes.toBytes("Pub111");
1881     byte [] col2 = Bytes.toBytes("Pub222");
1882 
1883 
1884     Put put = new Put(row1);
1885     put.add(family, col1, Bytes.toBytes(10L));
1886     region.put(put);
1887 
1888     put = new Put(row2);
1889     put.add(family, col1, Bytes.toBytes(15L));
1890     region.put(put);
1891 
1892     put = new Put(row3);
1893     put.add(family, col2, Bytes.toBytes(20L));
1894     region.put(put);
1895 
1896     put = new Put(row4);
1897     put.add(family, col2, Bytes.toBytes(30L));
1898     region.put(put);
1899 
1900     put = new Put(row5);
1901     put.add(family, col1, Bytes.toBytes(40L));
1902     region.put(put);
1903 
1904     Scan scan = new Scan(row3, row4);
1905     scan.setMaxVersions();
1906     scan.addColumn(family, col1);
1907     InternalScanner s = region.getScanner(scan);
1908 
1909     List<KeyValue> results = new ArrayList<KeyValue>();
1910     assertEquals(false, s.next(results));
1911     assertEquals(0, results.size());
1912   }
1913 
1914   public void testIncrementColumnValue_UpdatingInPlace() throws IOException {
1915     initHRegion(tableName, getName(), fam1);
1916 
1917     long value = 1L;
1918     long amount = 3L;
1919 
1920     Put put = new Put(row);
1921     put.add(fam1, qual1, Bytes.toBytes(value));
1922     region.put(put);
1923 
1924     long result = region.incrementColumnValue(row, fam1, qual1, amount, true);
1925 
1926     assertEquals(value+amount, result);
1927 
1928     Store store = region.getStore(fam1);
1929     // ICV removes any extra values floating around in there.
1930     assertEquals(1, store.memstore.kvset.size());
1931     assertTrue(store.memstore.snapshot.isEmpty());
1932 
1933     assertICV(row, fam1, qual1, value+amount);
1934   }
1935 
1936   public void testIncrementColumnValue_BumpSnapshot() throws IOException {
1937     ManualEnvironmentEdge mee = new ManualEnvironmentEdge();
1938     EnvironmentEdgeManagerTestHelper.injectEdge(mee);
1939     initHRegion(tableName, getName(), fam1);
1940 
1941     long value = 42L;
1942     long incr = 44L;
1943 
1944     // first put something in kvset, then snapshot it.
1945     Put put = new Put(row);
1946     put.add(fam1, qual1, Bytes.toBytes(value));
1947     region.put(put);
1948 
1949     // get the store in question:
1950     Store s = region.getStore(fam1);
1951     s.snapshot(); //bam
1952 
1953     // now increment:
1954     long newVal = region.incrementColumnValue(row, fam1, qual1,
1955         incr, false);
1956 
1957     assertEquals(value+incr, newVal);
1958 
1959     // get both versions:
1960     Get get = new Get(row);
1961     get.setMaxVersions();
1962     get.addColumn(fam1,qual1);
1963 
1964     Result r = region.get(get, null);
1965     assertEquals(2, r.size());
1966     KeyValue first = r.raw()[0];
1967     KeyValue second = r.raw()[1];
1968 
1969     assertTrue("ICV failed to upgrade timestamp",
1970         first.getTimestamp() != second.getTimestamp());
1971   }
1972 
1973   public void testIncrementColumnValue_ConcurrentFlush() throws IOException {
1974     initHRegion(tableName, getName(), fam1);
1975 
1976     long value = 1L;
1977     long amount = 3L;
1978 
1979     Put put = new Put(row);
1980     put.add(fam1, qual1, Bytes.toBytes(value));
1981     region.put(put);
1982 
1983     // now increment during a flush
1984     Thread t = new Thread() {
1985       public void run() {
1986         try {
1987           region.flushcache();
1988         } catch (IOException e) {
1989           LOG.info("test ICV, got IOE during flushcache()");
1990         }
1991       }
1992     };
1993     t.start();
1994     long r = region.incrementColumnValue(row, fam1, qual1, amount, true);
1995     assertEquals(value+amount, r);
1996 
1997     // this also asserts there is only 1 KeyValue in the set.
1998     assertICV(row, fam1, qual1, value+amount);
1999   }
2000 
2001   public void testIncrementColumnValue_heapSize() throws IOException {
2002     EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
2003 
2004     initHRegion(tableName, getName(), fam1);
2005 
2006     long byAmount = 1L;
2007     long size;
2008 
2009     for( int i = 0; i < 1000 ; i++) {
2010       region.incrementColumnValue(row, fam1, qual1, byAmount, true);
2011 
2012       size = region.memstoreSize.get();
2013       assertTrue("memstore size: " + size, size >= 0);
2014     }
2015   }
2016 
2017   public void testIncrementColumnValue_UpdatingInPlace_Negative()
2018     throws IOException {
2019     initHRegion(tableName, getName(), fam1);
2020 
2021     long value = 3L;
2022     long amount = -1L;
2023 
2024     Put put = new Put(row);
2025     put.add(fam1, qual1, Bytes.toBytes(value));
2026     region.put(put);
2027 
2028     long result = region.incrementColumnValue(row, fam1, qual1, amount, true);
2029     assertEquals(value+amount, result);
2030 
2031     assertICV(row, fam1, qual1, value+amount);
2032   }
2033 
2034   public void testIncrementColumnValue_AddingNew()
2035     throws IOException {
2036     initHRegion(tableName, getName(), fam1);
2037 
2038     long value = 1L;
2039     long amount = 3L;
2040 
2041     Put put = new Put(row);
2042     put.add(fam1, qual1, Bytes.toBytes(value));
2043     put.add(fam1, qual2, Bytes.toBytes(value));
2044     region.put(put);
2045 
2046     long result = region.incrementColumnValue(row, fam1, qual3, amount, true);
2047     assertEquals(amount, result);
2048 
2049     Get get = new Get(row);
2050     get.addColumn(fam1, qual3);
2051     Result rr = region.get(get, null);
2052     assertEquals(1, rr.size());
2053 
2054     // ensure none of the other cols were incremented.
2055     assertICV(row, fam1, qual1, value);
2056     assertICV(row, fam1, qual2, value);
2057     assertICV(row, fam1, qual3, amount);
2058   }
2059 
2060   public void testIncrementColumnValue_UpdatingFromSF() throws IOException {
2061     initHRegion(tableName, getName(), fam1);
2062 
2063     long value = 1L;
2064     long amount = 3L;
2065 
2066     Put put = new Put(row);
2067     put.add(fam1, qual1, Bytes.toBytes(value));
2068     put.add(fam1, qual2, Bytes.toBytes(value));
2069     region.put(put);
2070 
2071     // flush to disk.
2072     region.flushcache();
2073 
2074     Store store = region.getStore(fam1);
2075     assertEquals(0, store.memstore.kvset.size());
2076 
2077     long r = region.incrementColumnValue(row, fam1, qual1, amount, true);
2078     assertEquals(value+amount, r);
2079 
2080     assertICV(row, fam1, qual1, value+amount);
2081   }
2082 
2083   public void testIncrementColumnValue_AddingNewAfterSFCheck()
2084     throws IOException {
2085     initHRegion(tableName, getName(), fam1);
2086 
2087     long value = 1L;
2088     long amount = 3L;
2089 
2090     Put put = new Put(row);
2091     put.add(fam1, qual1, Bytes.toBytes(value));
2092     put.add(fam1, qual2, Bytes.toBytes(value));
2093     region.put(put);
2094     region.flushcache();
2095 
2096     Store store = region.getStore(fam1);
2097     assertEquals(0, store.memstore.kvset.size());
2098 
2099     long r = region.incrementColumnValue(row, fam1, qual3, amount, true);
2100     assertEquals(amount, r);
2101 
2102     assertICV(row, fam1, qual3, amount);
2103 
2104     region.flushcache();
2105 
2106     // ensure that this gets to disk.
2107     assertICV(row, fam1, qual3, amount);
2108   }
2109 
2110   /**
2111    * Added for HBASE-3235.
2112    *
2113    * When the initial put and an ICV update were arriving with the same timestamp,
2114    * the initial Put KV was being skipped during {@link MemStore#upsert(KeyValue)}
2115    * causing the iteration for matching KVs, causing the update-in-place to not
2116    * happen and the ICV put to effectively disappear.
2117    * @throws IOException
2118    */
2119   public void testIncrementColumnValue_UpdatingInPlace_TimestampClobber() throws IOException {
2120     initHRegion(tableName, getName(), fam1);
2121 
2122     long value = 1L;
2123     long amount = 3L;
2124     long now = EnvironmentEdgeManager.currentTimeMillis();
2125     ManualEnvironmentEdge mock = new ManualEnvironmentEdge();
2126     mock.setValue(now);
2127     EnvironmentEdgeManagerTestHelper.injectEdge(mock);
2128 
2129     // verify we catch an ICV on a put with the same timestamp
2130     Put put = new Put(row);
2131     put.add(fam1, qual1, now, Bytes.toBytes(value));
2132     region.put(put);
2133 
2134     long result = region.incrementColumnValue(row, fam1, qual1, amount, true);
2135 
2136     assertEquals(value+amount, result);
2137 
2138     Store store = region.getStore(fam1);
2139     // ICV should update the existing Put with the same timestamp
2140     assertEquals(1, store.memstore.kvset.size());
2141     assertTrue(store.memstore.snapshot.isEmpty());
2142 
2143     assertICV(row, fam1, qual1, value+amount);
2144 
2145     // verify we catch an ICV even when the put ts > now
2146     put = new Put(row);
2147     put.add(fam1, qual2, now+1, Bytes.toBytes(value));
2148     region.put(put);
2149 
2150     result = region.incrementColumnValue(row, fam1, qual2, amount, true);
2151 
2152     assertEquals(value+amount, result);
2153 
2154     store = region.getStore(fam1);
2155     // ICV should update the existing Put with the same timestamp
2156     assertEquals(2, store.memstore.kvset.size());
2157     assertTrue(store.memstore.snapshot.isEmpty());
2158 
2159     assertICV(row, fam1, qual2, value+amount);
2160     EnvironmentEdgeManagerTestHelper.reset();
2161   }
2162 
2163   public void testIncrementColumnValue_WrongInitialSize() throws IOException {
2164     initHRegion(tableName, getName(), fam1);
2165 
2166     byte[] row1 = Bytes.add(Bytes.toBytes("1234"), Bytes.toBytes(0L));
2167     int row1Field1 = 0;
2168     int row1Field2 = 1;
2169     Put put1 = new Put(row1);
2170     put1.add(fam1, qual1, Bytes.toBytes(row1Field1));
2171     put1.add(fam1, qual2, Bytes.toBytes(row1Field2));
2172     region.put(put1);
2173 
2174     long result;
2175     try {
2176         result = region.incrementColumnValue(row1, fam1, qual1, 1, true);
2177         fail("Expected to fail here");
2178     } catch (Exception exception) {
2179         // Expected.
2180     }
2181 
2182 
2183     assertICV(row1, fam1, qual1, row1Field1);
2184     assertICV(row1, fam1, qual2, row1Field2);
2185   }
2186 
2187   private void assertICV(byte [] row,
2188                          byte [] familiy,
2189                          byte[] qualifier,
2190                          long amount) throws IOException {
2191     // run a get and see?
2192     Get get = new Get(row);
2193     get.addColumn(familiy, qualifier);
2194     Result result = region.get(get, null);
2195     assertEquals(1, result.size());
2196 
2197     KeyValue kv = result.raw()[0];
2198     long r = Bytes.toLong(kv.getValue());
2199     assertEquals(amount, r);
2200   }
2201 
2202   private void assertICV(byte [] row,
2203                          byte [] familiy,
2204                          byte[] qualifier,
2205                          int amount) throws IOException {
2206     // run a get and see?
2207     Get get = new Get(row);
2208     get.addColumn(familiy, qualifier);
2209     Result result = region.get(get, null);
2210     assertEquals(1, result.size());
2211 
2212     KeyValue kv = result.raw()[0];
2213     int r = Bytes.toInt(kv.getValue());
2214     assertEquals(amount, r);
2215   }
2216 
2217   public void testScanner_Wildcard_FromMemStoreAndFiles_EnforceVersions()
2218   throws IOException {
2219     byte [] tableName = Bytes.toBytes("testtable");
2220     byte [] row1 = Bytes.toBytes("row1");
2221     byte [] fam1 = Bytes.toBytes("fam1");
2222     byte [] qf1 = Bytes.toBytes("qualifier1");
2223     byte [] qf2 = Bytes.toBytes("quateslifier2");
2224 
2225     long ts1 = 1;
2226     long ts2 = ts1 + 1;
2227     long ts3 = ts1 + 2;
2228     long ts4 = ts1 + 3;
2229 
2230     //Setting up region
2231     String method = this.getName();
2232     initHRegion(tableName, method, fam1);
2233 
2234     //Putting data in Region
2235     KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
2236     KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2237     KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2238     KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2239 
2240     KeyValue kv24 = new KeyValue(row1, fam1, qf2, ts4, KeyValue.Type.Put, null);
2241     KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2242     KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2243     KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2244 
2245     Put put = null;
2246     put = new Put(row1);
2247     put.add(kv14);
2248     put.add(kv24);
2249     region.put(put);
2250     region.flushcache();
2251 
2252     put = new Put(row1);
2253     put.add(kv23);
2254     put.add(kv13);
2255     region.put(put);
2256     region.flushcache();
2257 
2258     put = new Put(row1);
2259     put.add(kv22);
2260     put.add(kv12);
2261     region.put(put);
2262     region.flushcache();
2263 
2264     put = new Put(row1);
2265     put.add(kv21);
2266     put.add(kv11);
2267     region.put(put);
2268 
2269     //Expected
2270     List<KeyValue> expected = new ArrayList<KeyValue>();
2271     expected.add(kv14);
2272     expected.add(kv13);
2273     expected.add(kv12);
2274     expected.add(kv24);
2275     expected.add(kv23);
2276     expected.add(kv22);
2277 
2278     Scan scan = new Scan(row1);
2279     int versions = 3;
2280     scan.setMaxVersions(versions);
2281     List<KeyValue> actual = new ArrayList<KeyValue>();
2282     InternalScanner scanner = region.getScanner(scan);
2283 
2284     boolean hasNext = scanner.next(actual);
2285     assertEquals(false, hasNext);
2286 
2287     //Verify result
2288     for(int i=0; i<expected.size(); i++) {
2289       assertEquals(expected.get(i), actual.get(i));
2290     }
2291   }
2292 
2293   //////////////////////////////////////////////////////////////////////////////
2294   // Split test
2295   //////////////////////////////////////////////////////////////////////////////
2296   /**
2297    * Splits twice and verifies getting from each of the split regions.
2298    * @throws Exception
2299    */
2300   public void testBasicSplit() throws Exception {
2301     byte [] tableName = Bytes.toBytes("testtable");
2302     byte [][] families = {fam1, fam2, fam3};
2303 
2304     Configuration hc = initSplit();
2305     //Setting up region
2306     String method = this.getName();
2307     initHRegion(tableName, method, hc, families);
2308 
2309     try {
2310       LOG.info("" + addContent(region, fam3));
2311       region.flushcache();
2312       region.compactStores();
2313       byte [] splitRow = region.checkSplit();
2314       assertNotNull(splitRow);
2315       LOG.info("SplitRow: " + Bytes.toString(splitRow));
2316       HRegion [] regions = splitRegion(region, splitRow);
2317       try {
2318         // Need to open the regions.
2319         // TODO: Add an 'open' to HRegion... don't do open by constructing
2320         // instance.
2321         for (int i = 0; i < regions.length; i++) {
2322           regions[i] = openClosedRegion(regions[i]);
2323         }
2324         // Assert can get rows out of new regions. Should be able to get first
2325         // row from first region and the midkey from second region.
2326         assertGet(regions[0], fam3, Bytes.toBytes(START_KEY));
2327         assertGet(regions[1], fam3, splitRow);
2328         // Test I can get scanner and that it starts at right place.
2329         assertScan(regions[0], fam3,
2330             Bytes.toBytes(START_KEY));
2331         assertScan(regions[1], fam3, splitRow);
2332         // Now prove can't split regions that have references.
2333         for (int i = 0; i < regions.length; i++) {
2334           // Add so much data to this region, we create a store file that is >
2335           // than one of our unsplitable references. it will.
2336           for (int j = 0; j < 2; j++) {
2337             addContent(regions[i], fam3);
2338           }
2339           addContent(regions[i], fam2);
2340           addContent(regions[i], fam1);
2341           regions[i].flushcache();
2342         }
2343 
2344         byte [][] midkeys = new byte [regions.length][];
2345         // To make regions splitable force compaction.
2346         for (int i = 0; i < regions.length; i++) {
2347           regions[i].compactStores();
2348           midkeys[i] = regions[i].checkSplit();
2349         }
2350 
2351         TreeMap<String, HRegion> sortedMap = new TreeMap<String, HRegion>();
2352         // Split these two daughter regions so then I'll have 4 regions. Will
2353         // split because added data above.
2354         for (int i = 0; i < regions.length; i++) {
2355           HRegion[] rs = null;
2356           if (midkeys[i] != null) {
2357             rs = splitRegion(regions[i], midkeys[i]);
2358             for (int j = 0; j < rs.length; j++) {
2359               sortedMap.put(Bytes.toString(rs[j].getRegionName()),
2360                 openClosedRegion(rs[j]));
2361             }
2362           }
2363         }
2364         LOG.info("Made 4 regions");
2365         // The splits should have been even. Test I can get some arbitrary row
2366         // out of each.
2367         int interval = (LAST_CHAR - FIRST_CHAR) / 3;
2368         byte[] b = Bytes.toBytes(START_KEY);
2369         for (HRegion r : sortedMap.values()) {
2370           assertGet(r, fam3, b);
2371           b[0] += interval;
2372         }
2373       } finally {
2374         for (int i = 0; i < regions.length; i++) {
2375           try {
2376             regions[i].close();
2377           } catch (IOException e) {
2378             // Ignore.
2379           }
2380         }
2381       }
2382     } finally {
2383       if (region != null) {
2384         region.close();
2385         region.getLog().closeAndDelete();
2386       }
2387     }
2388   }
2389 
2390   public void testSplitRegion() throws IOException {
2391     byte [] tableName = Bytes.toBytes("testtable");
2392     byte [] qualifier = Bytes.toBytes("qualifier");
2393     Configuration hc = initSplit();
2394     int numRows = 10;
2395     byte [][] families = {fam1, fam3};
2396 
2397     //Setting up region
2398     String method = this.getName();
2399     initHRegion(tableName, method, hc, families);
2400 
2401     //Put data in region
2402     int startRow = 100;
2403     putData(startRow, numRows, qualifier, families);
2404     int splitRow = startRow + numRows;
2405     putData(splitRow, numRows, qualifier, families);
2406     region.flushcache();
2407 
2408     HRegion [] regions = null;
2409     try {
2410       regions = splitRegion(region, Bytes.toBytes("" + splitRow));
2411       //Opening the regions returned.
2412       for (int i = 0; i < regions.length; i++) {
2413         regions[i] = openClosedRegion(regions[i]);
2414       }
2415       //Verifying that the region has been split
2416       assertEquals(2, regions.length);
2417 
2418       //Verifying that all data is still there and that data is in the right
2419       //place
2420       verifyData(regions[0], startRow, numRows, qualifier, families);
2421       verifyData(regions[1], splitRow, numRows, qualifier, families);
2422 
2423     } finally {
2424       if (region != null) {
2425         region.close();
2426         region.getLog().closeAndDelete();
2427       }
2428     }
2429   }
2430 
2431 
2432   /**
2433    * Flushes the cache in a thread while scanning. The tests verify that the
2434    * scan is coherent - e.g. the returned results are always of the same or
2435    * later update as the previous results.
2436    * @throws IOException scan / compact
2437    * @throws InterruptedException thread join
2438    */
2439   public void testFlushCacheWhileScanning() throws IOException, InterruptedException {
2440     byte[] tableName = Bytes.toBytes("testFlushCacheWhileScanning");
2441     byte[] family = Bytes.toBytes("family");
2442     int numRows = 1000;
2443     int flushAndScanInterval = 10;
2444     int compactInterval = 10 * flushAndScanInterval;
2445 
2446     String method = "testFlushCacheWhileScanning";
2447     initHRegion(tableName,method, family);
2448     FlushThread flushThread = new FlushThread();
2449     flushThread.start();
2450 
2451     Scan scan = new Scan();
2452     scan.addFamily(family);
2453     scan.setFilter(new SingleColumnValueFilter(family, qual1,
2454       CompareOp.EQUAL, new BinaryComparator(Bytes.toBytes(5L))));
2455 
2456     int expectedCount = 0;
2457     List<KeyValue> res = new ArrayList<KeyValue>();
2458 
2459     boolean toggle=true;
2460     for (long i = 0; i < numRows; i++) {
2461       Put put = new Put(Bytes.toBytes(i));
2462       put.setWriteToWAL(false);
2463       put.add(family, qual1, Bytes.toBytes(i % 10));
2464       region.put(put);
2465 
2466       if (i != 0 && i % compactInterval == 0) {
2467         //System.out.println("iteration = " + i);
2468         region.compactStores(true);
2469       }
2470 
2471       if (i % 10 == 5L) {
2472         expectedCount++;
2473       }
2474 
2475       if (i != 0 && i % flushAndScanInterval == 0) {
2476         res.clear();
2477         InternalScanner scanner = region.getScanner(scan);
2478         if (toggle) {
2479           flushThread.flush();
2480         }
2481         while (scanner.next(res)) ;
2482         if (!toggle) {
2483           flushThread.flush();
2484         }
2485         assertEquals("i=" + i, expectedCount, res.size());
2486         toggle = !toggle;
2487       }
2488     }
2489 
2490     flushThread.done();
2491     flushThread.join();
2492     flushThread.checkNoError();
2493   }
2494 
2495   protected class FlushThread extends Thread {
2496     private volatile boolean done;
2497     private Throwable error = null;
2498 
2499     public void done() {
2500       done = true;
2501       synchronized (this) {
2502         interrupt();
2503       }
2504     }
2505 
2506     public void checkNoError() {
2507       if (error != null) {
2508         assertNull(error);
2509       }
2510     }
2511 
2512     @Override
2513     public void run() {
2514       done = false;
2515       while (!done) {
2516         synchronized (this) {
2517           try {
2518             wait();
2519           } catch (InterruptedException ignored) {
2520             if (done) {
2521               break;
2522             }
2523           }
2524         }
2525         try {
2526           region.flushcache();
2527         } catch (IOException e) {
2528           if (!done) {
2529             LOG.error("Error while flusing cache", e);
2530             error = e;
2531           }
2532           break;
2533         }
2534       }
2535 
2536     }
2537 
2538     public void flush() {
2539       synchronized (this) {
2540         notify();
2541       }
2542 
2543     }
2544   }
2545 
2546   /**
2547    * Writes very wide records and scans for the latest every time..
2548    * Flushes and compacts the region every now and then to keep things
2549    * realistic.
2550    *
2551    * @throws IOException          by flush / scan / compaction
2552    * @throws InterruptedException when joining threads
2553    */
2554   public void testWritesWhileScanning()
2555     throws IOException, InterruptedException {
2556     byte[] tableName = Bytes.toBytes("testWritesWhileScanning");
2557     int testCount = 100;
2558     int numRows = 1;
2559     int numFamilies = 10;
2560     int numQualifiers = 100;
2561     int flushInterval = 7;
2562     int compactInterval = 5 * flushInterval;
2563     byte[][] families = new byte[numFamilies][];
2564     for (int i = 0; i < numFamilies; i++) {
2565       families[i] = Bytes.toBytes("family" + i);
2566     }
2567     byte[][] qualifiers = new byte[numQualifiers][];
2568     for (int i = 0; i < numQualifiers; i++) {
2569       qualifiers[i] = Bytes.toBytes("qual" + i);
2570     }
2571 
2572     String method = "testWritesWhileScanning";
2573     initHRegion(tableName, method, families);
2574     PutThread putThread = new PutThread(numRows, families, qualifiers);
2575     putThread.start();
2576     putThread.waitForFirstPut();
2577 
2578     FlushThread flushThread = new FlushThread();
2579     flushThread.start();
2580 
2581     Scan scan = new Scan(Bytes.toBytes("row0"), Bytes.toBytes("row1"));
2582 //    scan.setFilter(new RowFilter(CompareFilter.CompareOp.EQUAL,
2583 //      new BinaryComparator(Bytes.toBytes("row0"))));
2584 
2585     int expectedCount = numFamilies * numQualifiers;
2586     List<KeyValue> res = new ArrayList<KeyValue>();
2587 
2588     long prevTimestamp = 0L;
2589     for (int i = 0; i < testCount; i++) {
2590 
2591       if (i != 0 && i % compactInterval == 0) {
2592         region.compactStores(true);
2593       }
2594 
2595       if (i != 0 && i % flushInterval == 0) {
2596         //System.out.println("flush scan iteration = " + i);
2597         flushThread.flush();
2598       }
2599 
2600       boolean previousEmpty = res.isEmpty();
2601       res.clear();
2602       InternalScanner scanner = region.getScanner(scan);
2603       while (scanner.next(res)) ;
2604       if (!res.isEmpty() || !previousEmpty || i > compactInterval) {
2605         assertEquals("i=" + i, expectedCount, res.size());
2606         long timestamp = res.get(0).getTimestamp();
2607         assertTrue("Timestamps were broke: " + timestamp + " prev: " + prevTimestamp,
2608             timestamp >= prevTimestamp);
2609         prevTimestamp = timestamp;
2610       }
2611     }
2612 
2613     putThread.done();
2614 
2615     region.flushcache();
2616 
2617     putThread.join();
2618     putThread.checkNoError();
2619 
2620     flushThread.done();
2621     flushThread.join();
2622     flushThread.checkNoError();
2623   }
2624 
2625   protected class PutThread extends Thread {
2626     private volatile boolean done;
2627     private volatile int numPutsFinished = 0;
2628 
2629     private Throwable error = null;
2630     private int numRows;
2631     private byte[][] families;
2632     private byte[][] qualifiers;
2633 
2634     private PutThread(int numRows, byte[][] families,
2635       byte[][] qualifiers) {
2636       this.numRows = numRows;
2637       this.families = families;
2638       this.qualifiers = qualifiers;
2639     }
2640 
2641     /**
2642      * Block until this thread has put at least one row.
2643      */
2644     public void waitForFirstPut() throws InterruptedException {
2645       // wait until put thread actually puts some data
2646       while (numPutsFinished == 0) {
2647         checkNoError();
2648         Thread.sleep(50);
2649       }
2650     }
2651 
2652     public void done() {
2653       done = true;
2654       synchronized (this) {
2655         interrupt();
2656       }
2657     }
2658 
2659     public void checkNoError() {
2660       if (error != null) {
2661         assertNull(error);
2662       }
2663     }
2664 
2665     @Override
2666     public void run() {
2667       done = false;
2668       while (!done) {
2669         try {
2670           for (int r = 0; r < numRows; r++) {
2671             byte[] row = Bytes.toBytes("row" + r);
2672             Put put = new Put(row);
2673             put.setWriteToWAL(false);
2674             byte[] value = Bytes.toBytes(String.valueOf(numPutsFinished));
2675             for (byte[] family : families) {
2676               for (byte[] qualifier : qualifiers) {
2677                 put.add(family, qualifier, (long) numPutsFinished, value);
2678               }
2679             }
2680 //            System.out.println("Putting of kvsetsize=" + put.size());
2681             region.put(put);
2682             numPutsFinished++;
2683             if (numPutsFinished > 0 && numPutsFinished % 47 == 0) {
2684               System.out.println("put iteration = " + numPutsFinished);
2685               Delete delete = new Delete(row, (long)numPutsFinished-30, null);
2686               region.delete(delete, null, true);
2687             }
2688             numPutsFinished++;
2689           }
2690         } catch (IOException e) {
2691           LOG.error("error while putting records", e);
2692           error = e;
2693           break;
2694         }
2695       }
2696 
2697     }
2698 
2699   }
2700 
2701 
2702   /**
2703    * Writes very wide records and gets the latest row every time..
2704    * Flushes and compacts the region every now and then to keep things
2705    * realistic.
2706    *
2707    * @throws IOException          by flush / scan / compaction
2708    * @throws InterruptedException when joining threads
2709    */
2710   public void testWritesWhileGetting()
2711     throws IOException, InterruptedException {
2712     byte[] tableName = Bytes.toBytes("testWritesWhileScanning");
2713     int testCount = 100;
2714     int numRows = 1;
2715     int numFamilies = 10;
2716     int numQualifiers = 100;
2717     int flushInterval = 10;
2718     int compactInterval = 10 * flushInterval;
2719     byte[][] families = new byte[numFamilies][];
2720     for (int i = 0; i < numFamilies; i++) {
2721       families[i] = Bytes.toBytes("family" + i);
2722     }
2723     byte[][] qualifiers = new byte[numQualifiers][];
2724     for (int i = 0; i < numQualifiers; i++) {
2725       qualifiers[i] = Bytes.toBytes("qual" + i);
2726     }
2727 
2728     String method = "testWritesWhileGetting";
2729     initHRegion(tableName, method, families);
2730     PutThread putThread = new PutThread(numRows, families, qualifiers);
2731     putThread.start();
2732     putThread.waitForFirstPut();
2733 
2734     FlushThread flushThread = new FlushThread();
2735     flushThread.start();
2736 
2737     Get get = new Get(Bytes.toBytes("row0"));
2738     Result result = null;
2739 
2740     int expectedCount = numFamilies * numQualifiers;
2741 
2742     long prevTimestamp = 0L;
2743     for (int i = 0; i < testCount; i++) {
2744 
2745       if (i != 0 && i % compactInterval == 0) {
2746         region.compactStores(true);
2747       }
2748 
2749       if (i != 0 && i % flushInterval == 0) {
2750         //System.out.println("iteration = " + i);
2751         flushThread.flush();
2752       }
2753 
2754       boolean previousEmpty = result == null || result.isEmpty();
2755       result = region.get(get, null);
2756       if (!result.isEmpty() || !previousEmpty || i > compactInterval) {
2757         assertEquals("i=" + i, expectedCount, result.size());
2758         // TODO this was removed, now what dangit?!
2759         // search looking for the qualifier in question?
2760         long timestamp = 0;
2761         for (KeyValue kv : result.raw()) {
2762           if (Bytes.equals(kv.getFamily(), families[0])
2763             && Bytes.equals(kv.getQualifier(), qualifiers[0])) {
2764             timestamp = kv.getTimestamp();
2765           }
2766         }
2767         assertTrue(timestamp >= prevTimestamp);
2768         prevTimestamp = timestamp;
2769         KeyValue previousKV = null;
2770 
2771         for (KeyValue kv : result.raw()) {
2772           byte[] thisValue = kv.getValue();
2773           if (previousKV != null) {
2774             if (Bytes.compareTo(previousKV.getValue(), thisValue) != 0) {
2775               LOG.warn("These two KV should have the same value." +
2776                 " Previous KV:" +
2777                 previousKV + "(memStoreTS:" + previousKV.getMemstoreTS() + ")" +
2778                 ", New KV: " +
2779                 kv + "(memStoreTS:" + kv.getMemstoreTS() + ")"
2780               );
2781               assertEquals(previousKV.getValue(), thisValue);
2782             }
2783           }
2784           previousKV = kv;
2785         }
2786       }
2787     }
2788 
2789     putThread.done();
2790 
2791     region.flushcache();
2792 
2793     putThread.join();
2794     putThread.checkNoError();
2795 
2796     flushThread.done();
2797     flushThread.join();
2798     flushThread.checkNoError();
2799   }
2800 
2801   public void testHolesInMeta() throws Exception {
2802     String method = "testHolesInMeta";
2803     byte[] tableName = Bytes.toBytes(method);
2804     byte[] family = Bytes.toBytes("family");
2805     initHRegion(tableName, Bytes.toBytes("x"), Bytes.toBytes("z"), method,
2806         HBaseConfiguration.create(), family);
2807     byte[] rowNotServed = Bytes.toBytes("a");
2808     Get g = new Get(rowNotServed);
2809     try {
2810       region.get(g, null);
2811       fail();
2812     } catch (WrongRegionException x) {
2813       // OK
2814     }
2815     byte[] row = Bytes.toBytes("y");
2816     g = new Get(row);
2817     region.get(g, null);
2818   }
2819 
2820   public void testSkipRecoveredEditsReplay() throws Exception {
2821     String method = "testSkipRecoveredEditsReplay";
2822     byte[] tableName = Bytes.toBytes(method);
2823     byte[] family = Bytes.toBytes("family");
2824     Configuration conf = HBaseConfiguration.create();
2825     initHRegion(tableName, method, conf, family);
2826     Path regiondir = region.getRegionDir();
2827     FileSystem fs = region.getFilesystem();
2828     byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
2829 
2830     Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
2831 
2832     long maxSeqId = 1050;
2833     long minSeqId = 1000;
2834 
2835     for (long i = minSeqId; i <= maxSeqId; i += 10) {
2836       Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
2837       HLog.Writer writer = HLog.createWriter(fs, recoveredEdits, conf);
2838 
2839       long time = System.nanoTime();
2840       WALEdit edit = new WALEdit();
2841       edit.add(new KeyValue(row, family, Bytes.toBytes(i),
2842           time, KeyValue.Type.Put, Bytes.toBytes(i)));
2843       writer.append(new HLog.Entry(new HLogKey(regionName, tableName,
2844           i, time, HConstants.DEFAULT_CLUSTER_ID), edit));
2845 
2846       writer.close();
2847     }
2848     MonitoredTask status = TaskMonitor.get().createStatus(method);
2849     long seqId = region.replayRecoveredEditsIfAny(regiondir, minSeqId-1, null, status);
2850     assertEquals(maxSeqId, seqId);
2851     Get get = new Get(row);
2852     Result result = region.get(get, null);
2853     for (long i = minSeqId; i <= maxSeqId; i += 10) {
2854       List<KeyValue> kvs = result.getColumn(family, Bytes.toBytes(i));
2855       assertEquals(1, kvs.size());
2856       assertEquals(Bytes.toBytes(i), kvs.get(0).getValue());
2857     }
2858   }
2859 
2860   public void testSkipRecoveredEditsReplaySomeIgnored() throws Exception {
2861     String method = "testSkipRecoveredEditsReplaySomeIgnored";
2862     byte[] tableName = Bytes.toBytes(method);
2863     byte[] family = Bytes.toBytes("family");
2864     initHRegion(tableName, method, HBaseConfiguration.create(), family);
2865     Path regiondir = region.getRegionDir();
2866     FileSystem fs = region.getFilesystem();
2867     byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
2868 
2869     Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
2870 
2871     long maxSeqId = 1050;
2872     long minSeqId = 1000;
2873 
2874     for (long i = minSeqId; i <= maxSeqId; i += 10) {
2875       Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
2876       HLog.Writer writer = HLog.createWriter(fs, recoveredEdits, conf);
2877 
2878       long time = System.nanoTime();
2879       WALEdit edit = new WALEdit();
2880       edit.add(new KeyValue(row, family, Bytes.toBytes(i),
2881           time, KeyValue.Type.Put, Bytes.toBytes(i)));
2882       writer.append(new HLog.Entry(new HLogKey(regionName, tableName,
2883           i, time, HConstants.DEFAULT_CLUSTER_ID), edit));
2884 
2885       writer.close();
2886     }
2887     long recoverSeqId = 1030;
2888     MonitoredTask status = TaskMonitor.get().createStatus(method);
2889     long seqId = region.replayRecoveredEditsIfAny(regiondir, recoverSeqId-1, null, status);
2890     assertEquals(maxSeqId, seqId);
2891     Get get = new Get(row);
2892     Result result = region.get(get, null);
2893     for (long i = minSeqId; i <= maxSeqId; i += 10) {
2894       List<KeyValue> kvs = result.getColumn(family, Bytes.toBytes(i));
2895       if (i < recoverSeqId) {
2896         assertEquals(0, kvs.size());
2897       } else {
2898         assertEquals(1, kvs.size());
2899         assertEquals(Bytes.toBytes(i), kvs.get(0).getValue());
2900       }
2901     }
2902   }
2903 
2904   public void testSkipRecoveredEditsReplayAllIgnored() throws Exception {
2905     String method = "testSkipRecoveredEditsReplayAllIgnored";
2906     byte[] tableName = Bytes.toBytes(method);
2907     byte[] family = Bytes.toBytes("family");
2908     initHRegion(tableName, method, HBaseConfiguration.create(), family);
2909     Path regiondir = region.getRegionDir();
2910     FileSystem fs = region.getFilesystem();
2911 
2912     Path recoveredEditsDir = HLog.getRegionDirRecoveredEditsDir(regiondir);
2913     for (int i = 1000; i < 1050; i += 10) {
2914       Path recoveredEdits = new Path(
2915           recoveredEditsDir, String.format("%019d", i));
2916       FSDataOutputStream dos=  fs.create(recoveredEdits);
2917       dos.writeInt(i);
2918       dos.close();
2919     }
2920     long minSeqId = 2000;
2921     Path recoveredEdits = new Path(
2922         recoveredEditsDir, String.format("%019d", minSeqId-1));
2923     FSDataOutputStream dos=  fs.create(recoveredEdits);
2924     dos.close();
2925     long seqId = region.replayRecoveredEditsIfAny(regiondir, minSeqId, null, null);
2926     assertEquals(minSeqId, seqId);
2927   }
2928 
2929   public void testIndexesScanWithOneDeletedRow() throws IOException {
2930     byte[] tableName = Bytes.toBytes("testIndexesScanWithOneDeletedRow");
2931     byte[] family = Bytes.toBytes("family");
2932 
2933     //Setting up region
2934     String method = "testIndexesScanWithOneDeletedRow";
2935     initHRegion(tableName, method, HBaseConfiguration.create(), family);
2936 
2937     Put put = new Put(Bytes.toBytes(1L));
2938     put.add(family, qual1, 1L, Bytes.toBytes(1L));
2939     region.put(put);
2940 
2941     region.flushcache();
2942 
2943     Delete delete = new Delete(Bytes.toBytes(1L), 1L, null);
2944     //delete.deleteColumn(family, qual1);
2945     region.delete(delete, null, true);
2946 
2947     put = new Put(Bytes.toBytes(2L));
2948     put.add(family, qual1, 2L, Bytes.toBytes(2L));
2949     region.put(put);
2950 
2951     Scan idxScan = new Scan();
2952     idxScan.addFamily(family);
2953     idxScan.setFilter(new FilterList(FilterList.Operator.MUST_PASS_ALL,
2954       Arrays.<Filter>asList(new SingleColumnValueFilter(family, qual1,
2955         CompareOp.GREATER_OR_EQUAL,
2956         new BinaryComparator(Bytes.toBytes(0L))),
2957         new SingleColumnValueFilter(family, qual1, CompareOp.LESS_OR_EQUAL,
2958           new BinaryComparator(Bytes.toBytes(3L)))
2959       )));
2960     InternalScanner scanner = region.getScanner(idxScan);
2961     List<KeyValue> res = new ArrayList<KeyValue>();
2962 
2963     //long start = System.nanoTime();
2964     while (scanner.next(res)) ;
2965     //long end = System.nanoTime();
2966     //System.out.println("memStoreEmpty=" + memStoreEmpty + ", time=" + (end - start)/1000000D);
2967     assertEquals(1L, res.size());
2968 
2969   }
2970 
2971   //////////////////////////////////////////////////////////////////////////////
2972   // Bloom filter test
2973   //////////////////////////////////////////////////////////////////////////////
2974   public void testBloomFilterSize() throws IOException {
2975     byte [] tableName = Bytes.toBytes("testBloomFilterSize");
2976     byte [] row1 = Bytes.toBytes("row1");
2977     byte [] fam1 = Bytes.toBytes("fam1");
2978     byte [] qf1  = Bytes.toBytes("col");
2979     byte [] val1  = Bytes.toBytes("value1");
2980     // Create Table
2981     HColumnDescriptor hcd = new HColumnDescriptor(fam1, Integer.MAX_VALUE,
2982         HColumnDescriptor.DEFAULT_COMPRESSION, false, true,
2983         HColumnDescriptor.DEFAULT_TTL, "rowcol");
2984 
2985     HTableDescriptor htd = new HTableDescriptor(tableName);
2986     htd.addFamily(hcd);
2987     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
2988     Path path = new Path(DIR + "testBloomFilterSize");
2989     region = HRegion.createHRegion(info, path, conf, htd);
2990 
2991     int num_unique_rows = 10;
2992     int duplicate_multiplier =2;
2993     int num_storefiles = 4;
2994 
2995     int version = 0;
2996     for (int f =0 ; f < num_storefiles; f++) {
2997       for (int i = 0; i < duplicate_multiplier; i ++) {
2998         for (int j = 0; j < num_unique_rows; j++) {
2999           Put put = new Put(Bytes.toBytes("row" + j));
3000           put.setWriteToWAL(false);
3001     			put.add(fam1, qf1, version++, val1);
3002           region.put(put);
3003         }
3004       }
3005       region.flushcache();
3006     }
3007     //before compaction
3008     Store store = region.getStore(fam1);
3009     List<StoreFile> storeFiles = store.getStorefiles();
3010     for (StoreFile storefile : storeFiles) {
3011       StoreFile.Reader reader = storefile.getReader();
3012       reader.loadFileInfo();
3013       reader.loadBloomfilter();
3014       assertEquals(num_unique_rows*duplicate_multiplier, reader.getEntries());
3015       assertEquals(num_unique_rows, reader.getFilterEntries());
3016     }
3017 
3018     region.compactStores(true);
3019 
3020     //after compaction
3021     storeFiles = store.getStorefiles();
3022     for (StoreFile storefile : storeFiles) {
3023       StoreFile.Reader reader = storefile.getReader();
3024       reader.loadFileInfo();
3025       reader.loadBloomfilter();
3026       assertEquals(num_unique_rows*duplicate_multiplier*num_storefiles,
3027           reader.getEntries());
3028       assertEquals(num_unique_rows, reader.getFilterEntries());
3029     }
3030   }
3031 
3032   public void testAllColumnsWithBloomFilter() throws IOException {
3033     byte [] TABLE = Bytes.toBytes("testAllColumnsWithBloomFilter");
3034     byte [] FAMILY = Bytes.toBytes("family");
3035 
3036     //Create table
3037     HColumnDescriptor hcd = new HColumnDescriptor(FAMILY, Integer.MAX_VALUE,
3038         HColumnDescriptor.DEFAULT_COMPRESSION,
3039         HColumnDescriptor.DEFAULT_IN_MEMORY,
3040         HColumnDescriptor.DEFAULT_BLOCKCACHE,
3041         HColumnDescriptor.DEFAULT_BLOCKSIZE, HColumnDescriptor.DEFAULT_TTL,
3042         "rowcol",
3043         HColumnDescriptor.DEFAULT_REPLICATION_SCOPE);
3044     HTableDescriptor htd = new HTableDescriptor(TABLE);
3045     htd.addFamily(hcd);
3046     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
3047     Path path = new Path(DIR + "testAllColumnsWithBloomFilter");
3048     region = HRegion.createHRegion(info, path, conf, htd);
3049 
3050     // For row:0, col:0: insert versions 1 through 5.
3051     byte row[] = Bytes.toBytes("row:" + 0);
3052     byte column[] = Bytes.toBytes("column:" + 0);
3053     Put put = new Put(row);
3054     put.setWriteToWAL(false);
3055     for (long idx = 1; idx <= 4; idx++) {
3056       put.add(FAMILY, column, idx, Bytes.toBytes("value-version-" + idx));
3057     }
3058     region.put(put);
3059 
3060     //Flush
3061     region.flushcache();
3062 
3063     //Get rows
3064     Get get = new Get(row);
3065     get.setMaxVersions();
3066     KeyValue[] kvs = region.get(get, null).raw();
3067 
3068     //Check if rows are correct
3069     assertEquals(4, kvs.length);
3070     checkOneCell(kvs[0], FAMILY, 0, 0, 4);
3071     checkOneCell(kvs[1], FAMILY, 0, 0, 3);
3072     checkOneCell(kvs[2], FAMILY, 0, 0, 2);
3073     checkOneCell(kvs[3], FAMILY, 0, 0, 1);
3074   }
3075 
3076   /**
3077     * Testcase to cover bug-fix for HBASE-2823
3078     * Ensures correct delete when issuing delete row
3079     * on columns with bloom filter set to row+col (BloomType.ROWCOL)
3080    */
3081   public void testDeleteRowWithBloomFilter() throws IOException {
3082     byte [] tableName = Bytes.toBytes("testDeleteRowWithBloomFilter");
3083     byte [] familyName = Bytes.toBytes("familyName");
3084 
3085     // Create Table
3086     HColumnDescriptor hcd = new HColumnDescriptor(familyName, Integer.MAX_VALUE,
3087         HColumnDescriptor.DEFAULT_COMPRESSION, false, true,
3088         HColumnDescriptor.DEFAULT_TTL, "rowcol");
3089 
3090     HTableDescriptor htd = new HTableDescriptor(tableName);
3091     htd.addFamily(hcd);
3092     HRegionInfo info = new HRegionInfo(htd.getName(), null, null, false);
3093     Path path = new Path(DIR + "TestDeleteRowWithBloomFilter");
3094     region = HRegion.createHRegion(info, path, conf, htd);
3095 
3096     // Insert some data
3097     byte row[] = Bytes.toBytes("row1");
3098     byte col[] = Bytes.toBytes("col1");
3099 
3100     Put put = new Put(row);
3101     put.add(familyName, col, 1, Bytes.toBytes("SomeRandomValue"));
3102     region.put(put);
3103     region.flushcache();
3104 
3105     Delete del = new Delete(row);
3106     region.delete(del, null, true);
3107     region.flushcache();
3108 
3109     // Get remaining rows (should have none)
3110     Get get = new Get(row);
3111     get.addColumn(familyName, col);
3112 
3113     KeyValue[] keyValues = region.get(get, null).raw();
3114     assertTrue(keyValues.length == 0);
3115   }
3116 
3117   @Test public void testgetHDFSBlocksDistribution() throws Exception {
3118     HBaseTestingUtility htu = new HBaseTestingUtility();
3119     final int DEFAULT_BLOCK_SIZE = 1024;
3120     htu.getConfiguration().setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
3121     htu.getConfiguration().setInt("dfs.replication", 2);
3122 
3123 
3124     // set up a cluster with 3 nodes
3125     MiniHBaseCluster cluster;
3126     String dataNodeHosts[] = new String[] { "host1", "host2", "host3" };
3127     int regionServersCount = 3;
3128 
3129     try {
3130       cluster = htu.startMiniCluster(1, regionServersCount, dataNodeHosts);
3131       byte [][] families = {fam1, fam2};
3132       HTable ht = htu.createTable(Bytes.toBytes(this.getName()), families);
3133 
3134       //Setting up region
3135       byte row[] = Bytes.toBytes("row1");
3136       byte col[] = Bytes.toBytes("col1");
3137 
3138       Put put = new Put(row);
3139       put.add(fam1, col, 1, Bytes.toBytes("test1"));
3140       put.add(fam2, col, 1, Bytes.toBytes("test2"));
3141       ht.put(put);
3142 
3143       HRegion firstRegion = htu.getHBaseCluster().
3144         getRegions(Bytes.toBytes(this.getName())).get(0);
3145       firstRegion.flushcache();
3146       HDFSBlocksDistribution blocksDistribution1 =
3147         firstRegion.getHDFSBlocksDistribution();
3148 
3149       // given the default replication factor is 2 and we have 2 HFiles,
3150       // we will have total of 4 replica of blocks on 3 datanodes; thus there
3151       // must be at least one host that have replica for 2 HFiles. That host's
3152       // weight will be equal to the unique block weight.
3153       long uniqueBlocksWeight1 =
3154         blocksDistribution1.getUniqueBlocksTotalWeight();
3155 
3156       String topHost = blocksDistribution1.getTopHosts().get(0);
3157       long topHostWeight = blocksDistribution1.getWeight(topHost);
3158       assertTrue(uniqueBlocksWeight1 == topHostWeight);
3159 
3160       // use the static method to compute the value, it should be the same.
3161       // static method is used by load balancer or other components
3162       HDFSBlocksDistribution blocksDistribution2 =
3163         HRegion.computeHDFSBlocksDistribution(htu.getConfiguration(),
3164         firstRegion.getTableDesc(),
3165         firstRegion.getRegionInfo().getEncodedName());
3166       long uniqueBlocksWeight2 =
3167         blocksDistribution2.getUniqueBlocksTotalWeight();
3168 
3169       assertTrue(uniqueBlocksWeight1 == uniqueBlocksWeight2);
3170       } finally {
3171         htu.shutdownMiniCluster();
3172       }
3173   }
3174 
3175   private void putData(int startRow, int numRows, byte [] qf,
3176       byte [] ...families)
3177   throws IOException {
3178     for(int i=startRow; i<startRow+numRows; i++) {
3179       Put put = new Put(Bytes.toBytes("" + i));
3180       put.setWriteToWAL(false);
3181       for(byte [] family : families) {
3182         put.add(family, qf, null);
3183       }
3184       region.put(put);
3185     }
3186   }
3187 
3188   private void verifyData(HRegion newReg, int startRow, int numRows, byte [] qf,
3189       byte [] ... families)
3190   throws IOException {
3191     for(int i=startRow; i<startRow + numRows; i++) {
3192       byte [] row = Bytes.toBytes("" + i);
3193       Get get = new Get(row);
3194       for(byte [] family : families) {
3195         get.addColumn(family, qf);
3196       }
3197       Result result = newReg.get(get, null);
3198       KeyValue [] raw = result.raw();
3199       assertEquals(families.length, result.size());
3200       for(int j=0; j<families.length; j++) {
3201         assertEquals(0, Bytes.compareTo(row, raw[j].getRow()));
3202         assertEquals(0, Bytes.compareTo(families[j], raw[j].getFamily()));
3203         assertEquals(0, Bytes.compareTo(qf, raw[j].getQualifier()));
3204       }
3205     }
3206   }
3207 
3208   private void assertGet(final HRegion r, final byte [] family, final byte [] k)
3209   throws IOException {
3210     // Now I have k, get values out and assert they are as expected.
3211     Get get = new Get(k).addFamily(family).setMaxVersions();
3212     KeyValue [] results = r.get(get, null).raw();
3213     for (int j = 0; j < results.length; j++) {
3214       byte [] tmp = results[j].getValue();
3215       // Row should be equal to value every time.
3216       assertTrue(Bytes.equals(k, tmp));
3217     }
3218   }
3219 
3220   /*
3221    * Assert first value in the passed region is <code>firstValue</code>.
3222    * @param r
3223    * @param fs
3224    * @param firstValue
3225    * @throws IOException
3226    */
3227   private void assertScan(final HRegion r, final byte [] fs,
3228       final byte [] firstValue)
3229   throws IOException {
3230     byte [][] families = {fs};
3231     Scan scan = new Scan();
3232     for (int i = 0; i < families.length; i++) scan.addFamily(families[i]);
3233     InternalScanner s = r.getScanner(scan);
3234     try {
3235       List<KeyValue> curVals = new ArrayList<KeyValue>();
3236       boolean first = true;
3237       OUTER_LOOP: while(s.next(curVals)) {
3238         for (KeyValue kv: curVals) {
3239           byte [] val = kv.getValue();
3240           byte [] curval = val;
3241           if (first) {
3242             first = false;
3243             assertTrue(Bytes.compareTo(curval, firstValue) == 0);
3244           } else {
3245             // Not asserting anything.  Might as well break.
3246             break OUTER_LOOP;
3247           }
3248         }
3249       }
3250     } finally {
3251       s.close();
3252     }
3253   }
3254 
3255   private Configuration initSplit() {
3256     Configuration conf = HBaseConfiguration.create();
3257     // Always compact if there is more than one store file.
3258     conf.setInt("hbase.hstore.compactionThreshold", 2);
3259 
3260     // Make lease timeout longer, lease checks less frequent
3261     conf.setInt("hbase.master.lease.thread.wakefrequency", 5 * 1000);
3262 
3263     conf.setInt(HConstants.HBASE_REGIONSERVER_LEASE_PERIOD_KEY, 10 * 1000);
3264 
3265     // Increase the amount of time between client retries
3266     conf.setLong("hbase.client.pause", 15 * 1000);
3267 
3268     // This size should make it so we always split using the addContent
3269     // below.  After adding all data, the first region is 1.3M
3270     conf.setLong("hbase.hregion.max.filesize", 1024 * 128);
3271     return conf;
3272   }
3273 
3274   private void initHRegion (byte [] tableName, String callingMethod,
3275     byte[] ... families)
3276   throws IOException {
3277     initHRegion(tableName, callingMethod, HBaseConfiguration.create(), families);
3278   }
3279 
3280   private void initHRegion (byte [] tableName, String callingMethod,
3281       Configuration conf, byte [] ... families)
3282     throws IOException{
3283     initHRegion(tableName, null, null, callingMethod, conf, families);
3284   }
3285 
3286   private void initHRegion(byte[] tableName, byte[] startKey, byte[] stopKey,
3287       String callingMethod, Configuration conf, byte[]... families)
3288       throws IOException {
3289     HTableDescriptor htd = new HTableDescriptor(tableName);
3290     for(byte [] family : families) {
3291       htd.addFamily(new HColumnDescriptor(family));
3292     }
3293     HRegionInfo info = new HRegionInfo(htd.getName(), startKey, stopKey, false);
3294     Path path = new Path(DIR + callingMethod);
3295     if (fs.exists(path)) {
3296       if (!fs.delete(path, true)) {
3297         throw new IOException("Failed delete of " + path);
3298       }
3299     }
3300     region = HRegion.createHRegion(info, path, conf, htd);
3301   }
3302 
3303   /**
3304    * Assert that the passed in KeyValue has expected contents for the
3305    * specified row, column & timestamp.
3306    */
3307   private void checkOneCell(KeyValue kv, byte[] cf,
3308                              int rowIdx, int colIdx, long ts) {
3309     String ctx = "rowIdx=" + rowIdx + "; colIdx=" + colIdx + "; ts=" + ts;
3310     assertEquals("Row mismatch which checking: " + ctx,
3311                  "row:"+ rowIdx, Bytes.toString(kv.getRow()));
3312     assertEquals("ColumnFamily mismatch while checking: " + ctx,
3313                  Bytes.toString(cf), Bytes.toString(kv.getFamily()));
3314     assertEquals("Column qualifier mismatch while checking: " + ctx,
3315                  "column:" + colIdx, Bytes.toString(kv.getQualifier()));
3316     assertEquals("Timestamp mismatch while checking: " + ctx,
3317                  ts, kv.getTimestamp());
3318     assertEquals("Value mismatch while checking: " + ctx,
3319                  "value-version-" + ts, Bytes.toString(kv.getValue()));
3320   }
3321 
3322 }