View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.regionserver;
20  
21  
22  import static org.apache.hadoop.hbase.HBaseTestingUtility.COLUMNS;
23  import static org.apache.hadoop.hbase.HBaseTestingUtility.FIRST_CHAR;
24  import static org.apache.hadoop.hbase.HBaseTestingUtility.LAST_CHAR;
25  import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY;
26  import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
27  import static org.apache.hadoop.hbase.HBaseTestingUtility.fam2;
28  import static org.apache.hadoop.hbase.HBaseTestingUtility.fam3;
29  import static org.junit.Assert.assertArrayEquals;
30  import static org.junit.Assert.assertEquals;
31  import static org.junit.Assert.assertFalse;
32  import static org.junit.Assert.assertNotNull;
33  import static org.junit.Assert.assertNull;
34  import static org.junit.Assert.assertTrue;
35  import static org.junit.Assert.fail;
36  import static org.mockito.Matchers.any;
37  import static org.mockito.Matchers.anyBoolean;
38  import static org.mockito.Matchers.anyLong;
39  import static org.mockito.Matchers.anyString;
40  import static org.mockito.Mockito.atLeast;
41  import static org.mockito.Mockito.doNothing;
42  import static org.mockito.Mockito.doReturn;
43  import static org.mockito.Mockito.doThrow;
44  import static org.mockito.Mockito.mock;
45  import static org.mockito.Mockito.never;
46  import static org.mockito.Mockito.spy;
47  import static org.mockito.Mockito.times;
48  import static org.mockito.Mockito.verify;
49  import static org.mockito.Mockito.when;
50  
51  import java.io.IOException;
52  import java.io.InterruptedIOException;
53  import java.security.PrivilegedExceptionAction;
54  import java.util.ArrayList;
55  import java.util.Arrays;
56  import java.util.Collection;
57  import java.util.Collections;
58  import java.util.HashMap;
59  import java.util.List;
60  import java.util.Map;
61  import java.util.NavigableMap;
62  import java.util.TreeMap;
63  import java.util.UUID;
64  import java.util.concurrent.Callable;
65  import java.util.concurrent.CountDownLatch;
66  import java.util.concurrent.ExecutorService;
67  import java.util.concurrent.Executors;
68  import java.util.concurrent.Future;
69  import java.util.concurrent.TimeUnit;
70  import java.util.concurrent.atomic.AtomicBoolean;
71  import java.util.concurrent.atomic.AtomicInteger;
72  import java.util.concurrent.atomic.AtomicReference;
73  
74  import org.apache.commons.lang.RandomStringUtils;
75  import org.apache.commons.logging.Log;
76  import org.apache.commons.logging.LogFactory;
77  import org.apache.hadoop.conf.Configuration;
78  import org.apache.hadoop.fs.FSDataOutputStream;
79  import org.apache.hadoop.fs.FileStatus;
80  import org.apache.hadoop.fs.FileSystem;
81  import org.apache.hadoop.fs.Path;
82  import org.apache.hadoop.hbase.Cell;
83  import org.apache.hadoop.hbase.CellComparator;
84  import org.apache.hadoop.hbase.CellUtil;
85  import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
86  import org.apache.hadoop.hbase.DroppedSnapshotException;
87  import org.apache.hadoop.hbase.HBaseConfiguration;
88  import org.apache.hadoop.hbase.HBaseTestCase;
89  import org.apache.hadoop.hbase.HBaseTestingUtility;
90  import org.apache.hadoop.hbase.HColumnDescriptor;
91  import org.apache.hadoop.hbase.HConstants;
92  import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
93  import org.apache.hadoop.hbase.HDFSBlocksDistribution;
94  import org.apache.hadoop.hbase.HRegionInfo;
95  import org.apache.hadoop.hbase.HTableDescriptor;
96  import org.apache.hadoop.hbase.KeyValue;
97  import org.apache.hadoop.hbase.KeyValueUtil;
98  import org.apache.hadoop.hbase.MiniHBaseCluster;
99  import org.apache.hadoop.hbase.MultithreadedTestUtil;
100 import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
101 import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
102 import org.apache.hadoop.hbase.NotServingRegionException;
103 import org.apache.hadoop.hbase.RegionTooBusyException;
104 import org.apache.hadoop.hbase.ServerName;
105 import org.apache.hadoop.hbase.TableName;
106 import org.apache.hadoop.hbase.Tag;
107 import org.apache.hadoop.hbase.TagType;
108 import org.apache.hadoop.hbase.Waiter;
109 import org.apache.hadoop.hbase.client.Append;
110 import org.apache.hadoop.hbase.client.Delete;
111 import org.apache.hadoop.hbase.client.Durability;
112 import org.apache.hadoop.hbase.client.Get;
113 import org.apache.hadoop.hbase.client.Increment;
114 import org.apache.hadoop.hbase.client.Mutation;
115 import org.apache.hadoop.hbase.client.Put;
116 import org.apache.hadoop.hbase.client.Result;
117 import org.apache.hadoop.hbase.client.RowMutations;
118 import org.apache.hadoop.hbase.client.Scan;
119 import org.apache.hadoop.hbase.client.Table;
120 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
121 import org.apache.hadoop.hbase.filter.BinaryComparator;
122 import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
123 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
124 import org.apache.hadoop.hbase.filter.Filter;
125 import org.apache.hadoop.hbase.filter.FilterBase;
126 import org.apache.hadoop.hbase.filter.FilterList;
127 import org.apache.hadoop.hbase.filter.NullComparator;
128 import org.apache.hadoop.hbase.filter.PrefixFilter;
129 import org.apache.hadoop.hbase.filter.SingleColumnValueExcludeFilter;
130 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
131 import org.apache.hadoop.hbase.io.hfile.HFile;
132 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
133 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
134 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
135 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
136 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
137 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
138 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
139 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
140 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
141 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
142 import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
143 import org.apache.hadoop.hbase.regionserver.Region.Operation;
144 import org.apache.hadoop.hbase.regionserver.Region.RowLock;
145 import org.apache.hadoop.hbase.regionserver.TestStore.FaultyFileSystem;
146 import org.apache.hadoop.hbase.regionserver.handler.FinishRegionRecoveringHandler;
147 import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
148 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
149 import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
150 import org.apache.hadoop.hbase.regionserver.wal.MetricsWALSource;
151 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
152 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
153 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
154 import org.apache.hadoop.hbase.security.User;
155 import org.apache.hadoop.hbase.test.MetricsAssertHelper;
156 import org.apache.hadoop.hbase.testclassification.MediumTests;
157 import org.apache.hadoop.hbase.util.Bytes;
158 import org.apache.hadoop.hbase.util.ClassSize;
159 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
160 import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
161 import org.apache.hadoop.hbase.util.FSUtils;
162 import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
163 import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
164 import org.apache.hadoop.hbase.util.PairOfSameType;
165 import org.apache.hadoop.hbase.util.Threads;
166 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
167 import org.apache.hadoop.hbase.wal.WAL;
168 import org.apache.hadoop.hbase.wal.WALFactory;
169 import org.apache.hadoop.hbase.wal.WALKey;
170 import org.apache.hadoop.hbase.wal.WALProvider;
171 import org.apache.hadoop.hbase.wal.WALProvider.Writer;
172 import org.apache.hadoop.hbase.wal.WALSplitter;
173 import org.junit.After;
174 import org.junit.Assert;
175 import org.junit.Before;
176 import org.junit.Rule;
177 import org.junit.Test;
178 import org.junit.experimental.categories.Category;
179 import org.junit.rules.TestName;
180 import org.mockito.ArgumentCaptor;
181 import org.mockito.ArgumentMatcher;
182 import org.mockito.Matchers;
183 import org.mockito.Mockito;
184 import org.mockito.invocation.InvocationOnMock;
185 import org.mockito.stubbing.Answer;
186 
187 import com.google.common.collect.ImmutableList;
188 import com.google.common.collect.Lists;
189 import com.google.common.collect.Maps;
190 import com.google.protobuf.ByteString;
191 
192 /**
193  * Basic stand-alone testing of HRegion.  No clusters!
194  *
195  * A lot of the meta information for an HRegion now lives inside other HRegions
196  * or in the HBaseMaster, so only basic testing is possible.
197  */
198 @Category(MediumTests.class)
199 @SuppressWarnings("deprecation")
200 public class TestHRegion {
201   // Do not spin up clusters in here. If you need to spin up a cluster, do it
202   // over in TestHRegionOnCluster.
203   private static final Log LOG = LogFactory.getLog(TestHRegion.class);
204   @Rule public TestName name = new TestName();
205 
206   private static final String COLUMN_FAMILY = "MyCF";
207   private static final byte [] COLUMN_FAMILY_BYTES = Bytes.toBytes(COLUMN_FAMILY);
208 
209   HRegion region = null;
210   // Do not run unit tests in parallel (? Why not?  It don't work?  Why not?  St.Ack)
211   private static HBaseTestingUtility TEST_UTIL;
212   public static Configuration CONF ;
213   private String dir;
214   private static FileSystem FILESYSTEM;
215   private final int MAX_VERSIONS = 2;
216 
217   // Test names
218   protected byte[] tableName;
219   protected String method;
220   protected final byte[] qual1 = Bytes.toBytes("qual1");
221   protected final byte[] qual2 = Bytes.toBytes("qual2");
222   protected final byte[] qual3 = Bytes.toBytes("qual3");
223   protected final byte[] value1 = Bytes.toBytes("value1");
224   protected final byte[] value2 = Bytes.toBytes("value2");
225   protected final byte[] row = Bytes.toBytes("rowA");
226   protected final byte[] row2 = Bytes.toBytes("rowB");
227 
228   protected final MetricsAssertHelper metricsAssertHelper = CompatibilitySingletonFactory
229       .getInstance(MetricsAssertHelper.class);
230 
231   @Before
232   public void setup() throws IOException {
233     TEST_UTIL = HBaseTestingUtility.createLocalHTU();
234     FILESYSTEM = TEST_UTIL.getTestFileSystem();
235     CONF = TEST_UTIL.getConfiguration();
236     dir = TEST_UTIL.getDataTestDir("TestHRegion").toString();
237     method = name.getMethodName();
238     tableName = Bytes.toBytes(name.getMethodName());
239   }
240 
241   @After
242   public void tearDown() throws Exception {
243     // Region may have been closed, but it is still no harm if we close it again here using HTU.
244     HBaseTestingUtility.closeRegionAndWAL(region);
245     EnvironmentEdgeManagerTestHelper.reset();
246     LOG.info("Cleaning test directory: " + TEST_UTIL.getDataTestDir());
247     TEST_UTIL.cleanupTestDir();
248   }
249 
250   String getName() {
251     return name.getMethodName();
252   }
253 
254   /**
255    * Test that I can use the max flushed sequence id after the close.
256    * @throws IOException
257    */
258   @Test (timeout = 100000)
259   public void testSequenceId() throws IOException {
260     region = initHRegion(tableName, name.getMethodName(), CONF, COLUMN_FAMILY_BYTES);
261     assertEquals(HConstants.NO_SEQNUM, region.getMaxFlushedSeqId());
262     // Weird. This returns 0 if no store files or no edits. Afraid to change it.
263     assertEquals(0, (long)region.getMaxStoreSeqId().get(COLUMN_FAMILY_BYTES));
264     HBaseTestingUtility.closeRegionAndWAL(this.region);
265     assertEquals(HConstants.NO_SEQNUM, region.getMaxFlushedSeqId());
266     assertEquals(0, (long)region.getMaxStoreSeqId().get(COLUMN_FAMILY_BYTES));
267     // Open region again.
268     region = initHRegion(tableName, name.getMethodName(), CONF, COLUMN_FAMILY_BYTES);
269     byte [] value = Bytes.toBytes(name.getMethodName());
270     // Make a random put against our cf.
271     Put put = new Put(value);
272     put.addColumn(COLUMN_FAMILY_BYTES, null, value);
273     region.put(put);
274     // No flush yet so init numbers should still be in place.
275     assertEquals(HConstants.NO_SEQNUM, region.getMaxFlushedSeqId());
276     assertEquals(0, (long)region.getMaxStoreSeqId().get(COLUMN_FAMILY_BYTES));
277     region.flush(true);
278     long max = region.getMaxFlushedSeqId();
279     HBaseTestingUtility.closeRegionAndWAL(this.region);
280     assertEquals(max, region.getMaxFlushedSeqId());
281     this.region = null;
282   }
283 
284   /**
285    * Test for Bug 2 of HBASE-10466.
286    * "Bug 2: Conditions for the first flush of region close (so-called pre-flush) If memstoreSize
287    * is smaller than a certain value, or when region close starts a flush is ongoing, the first
288    * flush is skipped and only the second flush takes place. However, two flushes are required in
289    * case previous flush fails and leaves some data in snapshot. The bug could cause loss of data
290    * in current memstore. The fix is removing all conditions except abort check so we ensure 2
291    * flushes for region close."
292    * @throws IOException
293    */
294   @Test (timeout=60000)
295   public void testCloseCarryingSnapshot() throws IOException {
296     region = initHRegion(tableName, name.getMethodName(), CONF, COLUMN_FAMILY_BYTES);
297     Store store = region.getStore(COLUMN_FAMILY_BYTES);
298     // Get some random bytes.
299     byte [] value = Bytes.toBytes(name.getMethodName());
300     // Make a random put against our cf.
301     Put put = new Put(value);
302     put.add(COLUMN_FAMILY_BYTES, null, value);
303     // First put something in current memstore, which will be in snapshot after flusher.prepare()
304     region.put(put);
305     StoreFlushContext storeFlushCtx = store.createFlushContext(12345);
306     storeFlushCtx.prepare();
307     // Second put something in current memstore
308     put.add(COLUMN_FAMILY_BYTES, Bytes.toBytes("abc"), value);
309     region.put(put);
310     // Close with something in memstore and something in the snapshot.  Make sure all is cleared.
311     HBaseTestingUtility.closeRegionAndWAL(region);
312     assertEquals(0, region.getMemstoreSize());
313     region = null;
314   }
315 
316   /**
317    * Test for HBASE-14229: Flushing canceled by coprocessor still leads to memstoreSize set down
318    */
319   @Test
320   public void testMemstoreSizeWithFlushCanceling() throws IOException {
321     FileSystem fs = FileSystem.get(CONF);
322     Path rootDir = new Path(dir + "testMemstoreSizeWithFlushCanceling");
323     FSHLog hLog = new FSHLog(fs, rootDir, "testMemstoreSizeWithFlushCanceling", CONF);
324     region = initHRegion(tableName, null, null, name.getMethodName(),
325         CONF, false, Durability.SYNC_WAL, hLog, COLUMN_FAMILY_BYTES);
326     Store store = region.getStore(COLUMN_FAMILY_BYTES);
327     assertEquals(0, region.getMemstoreSize());
328 
329     // Put some value and make sure flush could be completed normally
330     byte [] value = Bytes.toBytes(name.getMethodName());
331     Put put = new Put(value);
332     put.add(COLUMN_FAMILY_BYTES, Bytes.toBytes("abc"), value);
333     region.put(put);
334     long onePutSize = region.getMemstoreSize();
335     assertTrue(onePutSize > 0);
336     region.flush(true);
337     assertEquals("memstoreSize should be zero", 0, region.getMemstoreSize());
338     assertEquals("flushable size should be zero", 0, store.getFlushableSize());
339 
340     // save normalCPHost and replaced by mockedCPHost, which will cancel flush requests
341     RegionCoprocessorHost normalCPHost = region.getCoprocessorHost();
342     RegionCoprocessorHost mockedCPHost = Mockito.mock(RegionCoprocessorHost.class);
343     when(mockedCPHost.preFlush(Mockito.isA(HStore.class), Mockito.isA(InternalScanner.class))).
344       thenReturn(null);
345     region.setCoprocessorHost(mockedCPHost);
346     region.put(put);
347     region.flush(true);
348     assertEquals("memstoreSize should NOT be zero", onePutSize, region.getMemstoreSize());
349     assertEquals("flushable size should NOT be zero", onePutSize, store.getFlushableSize());
350 
351     // set normalCPHost and flush again, the snapshot will be flushed
352     region.setCoprocessorHost(normalCPHost);
353     region.flush(true);
354     assertEquals("memstoreSize should be zero", 0, region.getMemstoreSize());
355     assertEquals("flushable size should be zero", 0, store.getFlushableSize());
356   }
357 
358   @Test
359   public void testMemstoreSizeAccountingWithFailedPostBatchMutate() throws IOException {
360     String testName = "testMemstoreSizeAccountingWithFailedPostBatchMutate";
361     FileSystem fs = FileSystem.get(CONF);
362     Path rootDir = new Path(dir + testName);
363     FSHLog hLog = new FSHLog(fs, rootDir, testName, CONF);
364     region = initHRegion(tableName, null, null, name.getMethodName(),
365         CONF, false, Durability.SYNC_WAL, hLog, COLUMN_FAMILY_BYTES);
366     Store store = region.getStore(COLUMN_FAMILY_BYTES);
367     assertEquals(0, region.getMemstoreSize());
368 
369     // Put one value
370     byte [] value = Bytes.toBytes(name.getMethodName());
371     Put put = new Put(value);
372     put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("abc"), value);
373     region.put(put);
374     long onePutSize = region.getMemstoreSize();
375     assertTrue(onePutSize > 0);
376 
377     RegionCoprocessorHost mockedCPHost = Mockito.mock(RegionCoprocessorHost.class);
378     doThrow(new IOException())
379        .when(mockedCPHost).postBatchMutate(Mockito.<MiniBatchOperationInProgress<Mutation>>any());
380     region.setCoprocessorHost(mockedCPHost);
381 
382     put = new Put(value);
383     put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("dfg"), value);
384     try {
385       region.put(put);
386       fail("Should have failed with IOException");
387     } catch (IOException expected) {
388     }
389     long expectedSize = onePutSize * 2 - ClassSize.ARRAY;
390     assertEquals("memstoreSize should be incremented", expectedSize, region.getMemstoreSize());
391     assertEquals("flushable size should be incremented", expectedSize, store.getFlushableSize());
392 
393     region.setCoprocessorHost(null);
394   }
395 
396   /**
397    * Test we do not lose data if we fail a flush and then close.
398    * Part of HBase-10466.  Tests the following from the issue description:
399    * "Bug 1: Wrong calculation of HRegion.memstoreSize: When a flush fails, data to be flushed is
400    * kept in each MemStore's snapshot and wait for next flush attempt to continue on it. But when
401    * the next flush succeeds, the counter of total memstore size in HRegion is always deduced by
402    * the sum of current memstore sizes instead of snapshots left from previous failed flush. This
403    * calculation is problematic that almost every time there is failed flush, HRegion.memstoreSize
404    * gets reduced by a wrong value. If region flush could not proceed for a couple cycles, the size
405    * in current memstore could be much larger than the snapshot. It's likely to drift memstoreSize
406    * much smaller than expected. In extreme case, if the error accumulates to even bigger than
407    * HRegion's memstore size limit, any further flush is skipped because flush does not do anything
408    * if memstoreSize is not larger than 0."
409    * @throws Exception
410    */
411   @Test (timeout=60000)
412   public void testFlushSizeAccounting() throws Exception {
413     final Configuration conf = HBaseConfiguration.create(CONF);
414     // Only retry once.
415     conf.setInt("hbase.hstore.flush.retries.number", 1);
416     final User user =
417       User.createUserForTesting(conf, this.name.getMethodName(), new String[]{"foo"});
418     // Inject our faulty LocalFileSystem
419     conf.setClass("fs.file.impl", FaultyFileSystem.class, FileSystem.class);
420     user.runAs(new PrivilegedExceptionAction<Object>() {
421       @Override
422       public Object run() throws Exception {
423         // Make sure it worked (above is sensitive to caching details in hadoop core)
424         FileSystem fs = FileSystem.get(conf);
425         Assert.assertEquals(FaultyFileSystem.class, fs.getClass());
426         FaultyFileSystem ffs = (FaultyFileSystem)fs;
427         HRegion region = null;
428         try {
429           // Initialize region
430           region = initHRegion(tableName, name.getMethodName(), conf, COLUMN_FAMILY_BYTES);
431           long size = region.getMemstoreSize();
432           Assert.assertEquals(0, size);
433           // Put one item into memstore.  Measure the size of one item in memstore.
434           Put p1 = new Put(row);
435           p1.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual1, 1, (byte[])null));
436           region.put(p1);
437           final long sizeOfOnePut = region.getMemstoreSize();
438           // Fail a flush which means the current memstore will hang out as memstore 'snapshot'.
439           try {
440             LOG.info("Flushing");
441             region.flush(true);
442             Assert.fail("Didn't bubble up IOE!");
443           } catch (DroppedSnapshotException dse) {
444             // What we are expecting
445             region.closing.set(false); // this is needed for the rest of the test to work
446           } catch (Exception e) {
447             // What we are expecting
448             region.closing.set(false); // this is needed for the rest of the test to work
449           }
450           // Make it so all writes succeed from here on out
451           ffs.fault.set(false);
452           // WAL is bad because of above faulty fs. Roll WAL.
453           try {
454             region.getWAL().rollWriter(true);
455           } catch (Exception e) {
456             int x = 0;
457           }
458           // Check sizes.  Should still be the one entry.
459           Assert.assertEquals(sizeOfOnePut, region.getMemstoreSize());
460           // Now add two entries so that on this next flush that fails, we can see if we
461           // subtract the right amount, the snapshot size only.
462           Put p2 = new Put(row);
463           p2.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual2, 2, (byte[])null));
464           p2.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual3, 3, (byte[])null));
465           region.put(p2);
466           long expectedSize = sizeOfOnePut * 3- ClassSize.ARRAY;
467           Assert.assertEquals(expectedSize, region.getMemstoreSize());
468           // Do a successful flush.  It will clear the snapshot only.  Thats how flushes work.
469           // If already a snapshot, we clear it else we move the memstore to be snapshot and flush
470           // it
471           region.flush(true);
472           // Make sure our memory accounting is right.
473           Assert.assertEquals(sizeOfOnePut * 2 - ClassSize.ARRAY, region.getMemstoreSize());
474         } catch (Exception e) {
475           int x = 0;
476         } finally {
477           HRegion.closeHRegion(region);
478         }
479         return null;
480       }
481     });
482     FileSystem.closeAllForUGI(user.getUGI());
483   }
484 
485   @Test (timeout=60000)
486   public void testCloseWithFailingFlush() throws Exception {
487     final Configuration conf = HBaseConfiguration.create(CONF);
488     // Only retry once.
489     conf.setInt("hbase.hstore.flush.retries.number", 1);
490     final User user =
491       User.createUserForTesting(conf, this.name.getMethodName(), new String[]{"foo"});
492     // Inject our faulty LocalFileSystem
493     conf.setClass("fs.file.impl", FaultyFileSystem.class, FileSystem.class);
494     user.runAs(new PrivilegedExceptionAction<Object>() {
495       @Override
496       public Object run() throws Exception {
497         // Make sure it worked (above is sensitive to caching details in hadoop core)
498         FileSystem fs = FileSystem.get(conf);
499         Assert.assertEquals(FaultyFileSystem.class, fs.getClass());
500         FaultyFileSystem ffs = (FaultyFileSystem)fs;
501         HRegion region = null;
502         try {
503           // Initialize region
504           region = initHRegion(tableName, name.getMethodName(), conf, COLUMN_FAMILY_BYTES);
505           RegionServerServices services = mock(RegionServerServices.class);
506           doNothing().when(services).abort(anyString(), Matchers.<Throwable>any());
507           doReturn(ServerName.valueOf("fake-server", 0, 0L)). when(services).getServerName();
508           region.setRegionServerServices(services);
509 
510           long size = region.getMemstoreSize();
511           Assert.assertEquals(0, size);
512           // Put one item into memstore.  Measure the size of one item in memstore.
513           Put p1 = new Put(row);
514           p1.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual1, 1, (byte[])null));
515           region.put(p1);
516           // Manufacture an outstanding snapshot -- fake a failed flush by doing prepare step only.
517           Store store = region.getStore(COLUMN_FAMILY_BYTES);
518           StoreFlushContext storeFlushCtx = store.createFlushContext(12345);
519           storeFlushCtx.prepare();
520           // Now add two entries to the foreground memstore.
521           Put p2 = new Put(row);
522           p2.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual2, 2, (byte[])null));
523           p2.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual3, 3, (byte[])null));
524           region.put(p2);
525           // Now try close on top of a failing flush.
526           HBaseTestingUtility.closeRegionAndWAL(region);
527           region = null;
528           fail();
529         } catch (IOException dse) {
530           // Expected
531           LOG.info("Expected DroppedSnapshotException");
532         } finally {
533           // Make it so all writes succeed from here on out so can close clean
534           ffs.fault.set(false);
535           if (region != null) {
536             region.getWAL().rollWriter(true);
537             HRegion.closeHRegion(region);
538           }
539         }
540         return null;
541       }
542     });
543     FileSystem.closeAllForUGI(user.getUGI());
544   }
545 
546   @Test
547   public void testCompactionAffectedByScanners() throws Exception {
548     byte[] family = Bytes.toBytes("family");
549     this.region = initHRegion(tableName, method, CONF, family);
550 
551     Put put = new Put(Bytes.toBytes("r1"));
552     put.add(family, Bytes.toBytes("q1"), Bytes.toBytes("v1"));
553     region.put(put);
554     region.flush(true);
555 
556     Scan scan = new Scan();
557     scan.setMaxVersions(3);
558     // open the first scanner
559     RegionScanner scanner1 = region.getScanner(scan);
560 
561     Delete delete = new Delete(Bytes.toBytes("r1"));
562     region.delete(delete);
563     region.flush(true);
564 
565     // open the second scanner
566     RegionScanner scanner2 = region.getScanner(scan);
567 
568     List<Cell> results = new ArrayList<Cell>();
569 
570     System.out.println("Smallest read point:" + region.getSmallestReadPoint());
571 
572     // make a major compaction
573     region.compact(true);
574 
575     // open the third scanner
576     RegionScanner scanner3 = region.getScanner(scan);
577 
578     // get data from scanner 1, 2, 3 after major compaction
579     scanner1.next(results);
580     System.out.println(results);
581     assertEquals(1, results.size());
582 
583     results.clear();
584     scanner2.next(results);
585     System.out.println(results);
586     assertEquals(0, results.size());
587 
588     results.clear();
589     scanner3.next(results);
590     System.out.println(results);
591     assertEquals(0, results.size());
592   }
593 
594   @Test
595   public void testToShowNPEOnRegionScannerReseek() throws Exception {
596     byte[] family = Bytes.toBytes("family");
597     this.region = initHRegion(tableName, method, CONF, family);
598 
599     Put put = new Put(Bytes.toBytes("r1"));
600     put.add(family, Bytes.toBytes("q1"), Bytes.toBytes("v1"));
601     region.put(put);
602     put = new Put(Bytes.toBytes("r2"));
603     put.add(family, Bytes.toBytes("q1"), Bytes.toBytes("v1"));
604     region.put(put);
605     region.flush(true);
606 
607     Scan scan = new Scan();
608     scan.setMaxVersions(3);
609     // open the first scanner
610     RegionScanner scanner1 = region.getScanner(scan);
611 
612     System.out.println("Smallest read point:" + region.getSmallestReadPoint());
613 
614     region.compact(true);
615 
616     scanner1.reseek(Bytes.toBytes("r2"));
617     List<Cell> results = new ArrayList<Cell>();
618     scanner1.next(results);
619     Cell keyValue = results.get(0);
620     Assert.assertTrue(Bytes.compareTo(CellUtil.cloneRow(keyValue), Bytes.toBytes("r2")) == 0);
621     scanner1.close();
622   }
623 
624   @Test
625   public void testSkipRecoveredEditsReplay() throws Exception {
626     String method = "testSkipRecoveredEditsReplay";
627     TableName tableName = TableName.valueOf(method);
628     byte[] family = Bytes.toBytes("family");
629     this.region = initHRegion(tableName, method, CONF, family);
630     final WALFactory wals = new WALFactory(CONF, null, method);
631     try {
632       Path regiondir = region.getRegionFileSystem().getRegionDir();
633       FileSystem fs = region.getRegionFileSystem().getFileSystem();
634       byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
635 
636       Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir);
637 
638       long maxSeqId = 1050;
639       long minSeqId = 1000;
640 
641       for (long i = minSeqId; i <= maxSeqId; i += 10) {
642         Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
643         fs.create(recoveredEdits);
644         WALProvider.Writer writer = wals.createRecoveredEditsWriter(fs, recoveredEdits);
645 
646         long time = System.nanoTime();
647         WALEdit edit = new WALEdit();
648         edit.add(new KeyValue(row, family, Bytes.toBytes(i), time, KeyValue.Type.Put, Bytes
649             .toBytes(i)));
650         writer.append(new WAL.Entry(new HLogKey(regionName, tableName, i, time,
651             HConstants.DEFAULT_CLUSTER_ID), edit));
652 
653         writer.close();
654       }
655       MonitoredTask status = TaskMonitor.get().createStatus(method);
656       Map<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
657       for (Store store : region.getStores()) {
658         maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), minSeqId - 1);
659       }
660       long seqId = region.replayRecoveredEditsIfAny(maxSeqIdInStores, null, status);
661       assertEquals(maxSeqId, seqId);
662       region.getMVCC().advanceTo(seqId);
663       Get get = new Get(row);
664       Result result = region.get(get);
665       for (long i = minSeqId; i <= maxSeqId; i += 10) {
666         List<Cell> kvs = result.getColumnCells(family, Bytes.toBytes(i));
667         assertEquals(1, kvs.size());
668         assertArrayEquals(Bytes.toBytes(i), CellUtil.cloneValue(kvs.get(0)));
669       }
670     } finally {
671       HRegion.closeHRegion(this.region);
672       this.region = null;
673       wals.close();
674     }
675   }
676 
677   @Test
678   public void testSkipRecoveredEditsReplaySomeIgnored() throws Exception {
679     String method = "testSkipRecoveredEditsReplaySomeIgnored";
680     TableName tableName = TableName.valueOf(method);
681     byte[] family = Bytes.toBytes("family");
682     this.region = initHRegion(tableName, method, CONF, family);
683     final WALFactory wals = new WALFactory(CONF, null, method);
684     try {
685       Path regiondir = region.getRegionFileSystem().getRegionDir();
686       FileSystem fs = region.getRegionFileSystem().getFileSystem();
687       byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
688 
689       Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir);
690 
691       long maxSeqId = 1050;
692       long minSeqId = 1000;
693 
694       for (long i = minSeqId; i <= maxSeqId; i += 10) {
695         Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
696         fs.create(recoveredEdits);
697         WALProvider.Writer writer = wals.createRecoveredEditsWriter(fs, recoveredEdits);
698 
699         long time = System.nanoTime();
700         WALEdit edit = new WALEdit();
701         edit.add(new KeyValue(row, family, Bytes.toBytes(i), time, KeyValue.Type.Put, Bytes
702             .toBytes(i)));
703         writer.append(new WAL.Entry(new HLogKey(regionName, tableName, i, time,
704             HConstants.DEFAULT_CLUSTER_ID), edit));
705 
706         writer.close();
707       }
708       long recoverSeqId = 1030;
709       MonitoredTask status = TaskMonitor.get().createStatus(method);
710       Map<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
711       for (Store store : region.getStores()) {
712         maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), recoverSeqId - 1);
713       }
714       long seqId = region.replayRecoveredEditsIfAny(maxSeqIdInStores, null, status);
715       assertEquals(maxSeqId, seqId);
716       region.getMVCC().advanceTo(seqId);
717       Get get = new Get(row);
718       Result result = region.get(get);
719       for (long i = minSeqId; i <= maxSeqId; i += 10) {
720         List<Cell> kvs = result.getColumnCells(family, Bytes.toBytes(i));
721         if (i < recoverSeqId) {
722           assertEquals(0, kvs.size());
723         } else {
724           assertEquals(1, kvs.size());
725           assertArrayEquals(Bytes.toBytes(i), CellUtil.cloneValue(kvs.get(0)));
726         }
727       }
728     } finally {
729       HRegion.closeHRegion(this.region);
730       this.region = null;
731       wals.close();
732     }
733   }
734 
735   @Test
736   public void testSkipRecoveredEditsReplayAllIgnored() throws Exception {
737     byte[] family = Bytes.toBytes("family");
738     this.region = initHRegion(tableName, method, CONF, family);
739     Path regiondir = region.getRegionFileSystem().getRegionDir();
740     FileSystem fs = region.getRegionFileSystem().getFileSystem();
741 
742     Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir);
743     for (int i = 1000; i < 1050; i += 10) {
744       Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
745       FSDataOutputStream dos = fs.create(recoveredEdits);
746       dos.writeInt(i);
747       dos.close();
748     }
749     long minSeqId = 2000;
750     Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", minSeqId - 1));
751     FSDataOutputStream dos = fs.create(recoveredEdits);
752     dos.close();
753 
754     Map<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
755     for (Store store : region.getStores()) {
756       maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), minSeqId);
757     }
758     long seqId = region.replayRecoveredEditsIfAny(maxSeqIdInStores, null, null);
759     assertEquals(minSeqId, seqId);
760   }
761 
762   @Test
763   public void testSkipRecoveredEditsReplayTheLastFileIgnored() throws Exception {
764     String method = "testSkipRecoveredEditsReplayTheLastFileIgnored";
765     TableName tableName = TableName.valueOf(method);
766     byte[] family = Bytes.toBytes("family");
767     this.region = initHRegion(tableName, method, CONF, family);
768     final WALFactory wals = new WALFactory(CONF, null, method);
769     try {
770       Path regiondir = region.getRegionFileSystem().getRegionDir();
771       FileSystem fs = region.getRegionFileSystem().getFileSystem();
772       byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
773       byte[][] columns = region.getTableDesc().getFamiliesKeys().toArray(new byte[0][]);
774 
775       assertEquals(0, region.getStoreFileList(columns).size());
776 
777       Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir);
778 
779       long maxSeqId = 1050;
780       long minSeqId = 1000;
781 
782       for (long i = minSeqId; i <= maxSeqId; i += 10) {
783         Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
784         fs.create(recoveredEdits);
785         WALProvider.Writer writer = wals.createRecoveredEditsWriter(fs, recoveredEdits);
786 
787         long time = System.nanoTime();
788         WALEdit edit = null;
789         if (i == maxSeqId) {
790           edit = WALEdit.createCompaction(region.getRegionInfo(),
791           CompactionDescriptor.newBuilder()
792           .setTableName(ByteString.copyFrom(tableName.getName()))
793           .setFamilyName(ByteString.copyFrom(regionName))
794           .setEncodedRegionName(ByteString.copyFrom(regionName))
795           .setStoreHomeDirBytes(ByteString.copyFrom(Bytes.toBytes(regiondir.toString())))
796           .setRegionName(ByteString.copyFrom(region.getRegionInfo().getRegionName()))
797           .build());
798         } else {
799           edit = new WALEdit();
800           edit.add(new KeyValue(row, family, Bytes.toBytes(i), time, KeyValue.Type.Put, Bytes
801             .toBytes(i)));
802         }
803         writer.append(new WAL.Entry(new HLogKey(regionName, tableName, i, time,
804             HConstants.DEFAULT_CLUSTER_ID), edit));
805         writer.close();
806       }
807 
808       long recoverSeqId = 1030;
809       Map<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
810       MonitoredTask status = TaskMonitor.get().createStatus(method);
811       for (Store store : region.getStores()) {
812         maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), recoverSeqId - 1);
813       }
814       long seqId = region.replayRecoveredEditsIfAny(maxSeqIdInStores, null, status);
815       assertEquals(maxSeqId, seqId);
816 
817       // assert that the files are flushed
818       assertEquals(1, region.getStoreFileList(columns).size());
819 
820     } finally {
821       HRegion.closeHRegion(this.region);
822       this.region = null;
823       wals.close();
824     }
825   }
826 
827   @Test
828   public void testRecoveredEditsReplayCompaction() throws Exception {
829     testRecoveredEditsReplayCompaction(false);
830     testRecoveredEditsReplayCompaction(true);
831   }
832   public void testRecoveredEditsReplayCompaction(boolean mismatchedRegionName) throws Exception {
833     String method = name.getMethodName();
834     TableName tableName = TableName.valueOf(method);
835     byte[] family = Bytes.toBytes("family");
836     this.region = initHRegion(tableName, method, CONF, family);
837     final WALFactory wals = new WALFactory(CONF, null, method);
838     try {
839       Path regiondir = region.getRegionFileSystem().getRegionDir();
840       FileSystem fs = region.getRegionFileSystem().getFileSystem();
841       byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
842 
843       long maxSeqId = 3;
844       long minSeqId = 0;
845 
846       for (long i = minSeqId; i < maxSeqId; i++) {
847         Put put = new Put(Bytes.toBytes(i));
848         put.add(family, Bytes.toBytes(i), Bytes.toBytes(i));
849         region.put(put);
850         region.flush(true);
851       }
852 
853       // this will create a region with 3 files
854       assertEquals(3, region.getStore(family).getStorefilesCount());
855       List<Path> storeFiles = new ArrayList<Path>(3);
856       for (StoreFile sf : region.getStore(family).getStorefiles()) {
857         storeFiles.add(sf.getPath());
858       }
859 
860       // disable compaction completion
861       CONF.setBoolean("hbase.hstore.compaction.complete", false);
862       region.compactStores();
863 
864       // ensure that nothing changed
865       assertEquals(3, region.getStore(family).getStorefilesCount());
866 
867       // now find the compacted file, and manually add it to the recovered edits
868       Path tmpDir = new Path(region.getRegionFileSystem().getTempDir(), Bytes.toString(family));
869       FileStatus[] files = FSUtils.listStatus(fs, tmpDir);
870       String errorMsg = "Expected to find 1 file in the region temp directory "
871           + "from the compaction, could not find any";
872       assertNotNull(errorMsg, files);
873       assertEquals(errorMsg, 1, files.length);
874       // move the file inside region dir
875       Path newFile = region.getRegionFileSystem().commitStoreFile(Bytes.toString(family),
876           files[0].getPath());
877 
878       byte[] encodedNameAsBytes = this.region.getRegionInfo().getEncodedNameAsBytes();
879       byte[] fakeEncodedNameAsBytes = new byte [encodedNameAsBytes.length];
880       for (int i=0; i < encodedNameAsBytes.length; i++) {
881         // Mix the byte array to have a new encodedName
882         fakeEncodedNameAsBytes[i] = (byte) (encodedNameAsBytes[i] + 1);
883       }
884 
885       CompactionDescriptor compactionDescriptor = ProtobufUtil.toCompactionDescriptor(this.region
886         .getRegionInfo(), mismatchedRegionName ? fakeEncodedNameAsBytes : null, family,
887             storeFiles, Lists.newArrayList(newFile),
888             region.getRegionFileSystem().getStoreDir(Bytes.toString(family)));
889 
890       WALUtil.writeCompactionMarker(region.getWAL(), this.region.getTableDesc(),
891           this.region.getRegionInfo(), compactionDescriptor, region.getMVCC());
892 
893       Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir);
894 
895       Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", 1000));
896       fs.create(recoveredEdits);
897       WALProvider.Writer writer = wals.createRecoveredEditsWriter(fs, recoveredEdits);
898 
899       long time = System.nanoTime();
900 
901       writer.append(new WAL.Entry(new HLogKey(regionName, tableName, 10, time,
902           HConstants.DEFAULT_CLUSTER_ID), WALEdit.createCompaction(region.getRegionInfo(),
903           compactionDescriptor)));
904       writer.close();
905 
906       // close the region now, and reopen again
907       region.getTableDesc();
908       region.getRegionInfo();
909       HBaseTestingUtility.closeRegionAndWAL(this.region);
910       try {
911         region = HRegion.openHRegion(region, null);
912       } catch (WrongRegionException wre) {
913         fail("Matching encoded region name should not have produced WrongRegionException");
914       }
915 
916       // now check whether we have only one store file, the compacted one
917       Collection<StoreFile> sfs = region.getStore(family).getStorefiles();
918       for (StoreFile sf : sfs) {
919         LOG.info(sf.getPath());
920       }
921       if (!mismatchedRegionName) {
922         assertEquals(1, region.getStore(family).getStorefilesCount());
923       }
924       files = FSUtils.listStatus(fs, tmpDir);
925       assertTrue("Expected to find 0 files inside " + tmpDir, files == null || files.length == 0);
926 
927       for (long i = minSeqId; i < maxSeqId; i++) {
928         Get get = new Get(Bytes.toBytes(i));
929         Result result = region.get(get);
930         byte[] value = result.getValue(family, Bytes.toBytes(i));
931         assertArrayEquals(Bytes.toBytes(i), value);
932       }
933     } finally {
934       HRegion.closeHRegion(this.region);
935       this.region = null;
936       wals.close();
937     }
938   }
939 
940   @Test
941   public void testFlushMarkers() throws Exception {
942     // tests that flush markers are written to WAL and handled at recovered edits
943     String method = name.getMethodName();
944     TableName tableName = TableName.valueOf(method);
945     byte[] family = Bytes.toBytes("family");
946     Path logDir = TEST_UTIL.getDataTestDirOnTestFS(method + ".log");
947     final Configuration walConf = new Configuration(TEST_UTIL.getConfiguration());
948     FSUtils.setRootDir(walConf, logDir);
949     final WALFactory wals = new WALFactory(walConf, null, method);
950     final WAL wal = wals.getWAL(tableName.getName(), tableName.getNamespace());
951 
952     this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW,
953       HConstants.EMPTY_END_ROW, method, CONF, false, Durability.USE_DEFAULT, wal, family);
954     try {
955       Path regiondir = region.getRegionFileSystem().getRegionDir();
956       FileSystem fs = region.getRegionFileSystem().getFileSystem();
957       byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
958 
959       long maxSeqId = 3;
960       long minSeqId = 0;
961 
962       for (long i = minSeqId; i < maxSeqId; i++) {
963         Put put = new Put(Bytes.toBytes(i));
964         put.add(family, Bytes.toBytes(i), Bytes.toBytes(i));
965         region.put(put);
966         region.flush(true);
967       }
968 
969       // this will create a region with 3 files from flush
970       assertEquals(3, region.getStore(family).getStorefilesCount());
971       List<String> storeFiles = new ArrayList<String>(3);
972       for (StoreFile sf : region.getStore(family).getStorefiles()) {
973         storeFiles.add(sf.getPath().getName());
974       }
975 
976       // now verify that the flush markers are written
977       wal.shutdown();
978       WAL.Reader reader = WALFactory.createReader(fs, DefaultWALProvider.getCurrentFileName(wal),
979         TEST_UTIL.getConfiguration());
980       try {
981         List<WAL.Entry> flushDescriptors = new ArrayList<WAL.Entry>();
982         long lastFlushSeqId = -1;
983         while (true) {
984           WAL.Entry entry = reader.next();
985           if (entry == null) {
986             break;
987           }
988           Cell cell = entry.getEdit().getCells().get(0);
989           if (WALEdit.isMetaEditFamily(cell)) {
990             FlushDescriptor flushDesc = WALEdit.getFlushDescriptor(cell);
991             assertNotNull(flushDesc);
992             assertArrayEquals(tableName.getName(), flushDesc.getTableName().toByteArray());
993             if (flushDesc.getAction() == FlushAction.START_FLUSH) {
994               assertTrue(flushDesc.getFlushSequenceNumber() > lastFlushSeqId);
995             } else if (flushDesc.getAction() == FlushAction.COMMIT_FLUSH) {
996               assertTrue(flushDesc.getFlushSequenceNumber() == lastFlushSeqId);
997             }
998             lastFlushSeqId = flushDesc.getFlushSequenceNumber();
999             assertArrayEquals(regionName, flushDesc.getEncodedRegionName().toByteArray());
1000             assertEquals(1, flushDesc.getStoreFlushesCount()); //only one store
1001             StoreFlushDescriptor storeFlushDesc = flushDesc.getStoreFlushes(0);
1002             assertArrayEquals(family, storeFlushDesc.getFamilyName().toByteArray());
1003             assertEquals("family", storeFlushDesc.getStoreHomeDir());
1004             if (flushDesc.getAction() == FlushAction.START_FLUSH) {
1005               assertEquals(0, storeFlushDesc.getFlushOutputCount());
1006             } else {
1007               assertEquals(1, storeFlushDesc.getFlushOutputCount()); //only one file from flush
1008               assertTrue(storeFiles.contains(storeFlushDesc.getFlushOutput(0)));
1009             }
1010 
1011             flushDescriptors.add(entry);
1012           }
1013         }
1014 
1015         assertEquals(3 * 2, flushDescriptors.size()); // START_FLUSH and COMMIT_FLUSH per flush
1016 
1017         // now write those markers to the recovered edits again.
1018 
1019         Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir);
1020 
1021         Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", 1000));
1022         fs.create(recoveredEdits);
1023         WALProvider.Writer writer = wals.createRecoveredEditsWriter(fs, recoveredEdits);
1024 
1025         for (WAL.Entry entry : flushDescriptors) {
1026           writer.append(entry);
1027         }
1028         writer.close();
1029       } finally {
1030         if (null != reader) {
1031           try {
1032             reader.close();
1033           } catch (IOException exception) {
1034             LOG.warn("Problem closing wal: " + exception.getMessage());
1035             LOG.debug("exception details", exception);
1036           }
1037         }
1038       }
1039 
1040       // close the region now, and reopen again
1041       HBaseTestingUtility.closeRegionAndWAL(this.region);
1042       region = HRegion.openHRegion(region, null);
1043 
1044       // now check whether we have can read back the data from region
1045       for (long i = minSeqId; i < maxSeqId; i++) {
1046         Get get = new Get(Bytes.toBytes(i));
1047         Result result = region.get(get);
1048         byte[] value = result.getValue(family, Bytes.toBytes(i));
1049         assertArrayEquals(Bytes.toBytes(i), value);
1050       }
1051     } finally {
1052       HBaseTestingUtility.closeRegionAndWAL(this.region);
1053       this.region = null;
1054       wals.close();
1055     }
1056   }
1057 
1058   class IsFlushWALMarker extends ArgumentMatcher<WALEdit> {
1059     volatile FlushAction[] actions;
1060     public IsFlushWALMarker(FlushAction... actions) {
1061       this.actions = actions;
1062     }
1063     @Override
1064     public boolean matches(Object edit) {
1065       List<Cell> cells = ((WALEdit)edit).getCells();
1066       if (cells.isEmpty()) {
1067         return false;
1068       }
1069       if (WALEdit.isMetaEditFamily(cells.get(0))) {
1070         FlushDescriptor desc = null;
1071         try {
1072           desc = WALEdit.getFlushDescriptor(cells.get(0));
1073         } catch (IOException e) {
1074           LOG.warn(e);
1075           return false;
1076         }
1077         if (desc != null) {
1078           for (FlushAction action : actions) {
1079             if (desc.getAction() == action) {
1080               return true;
1081             }
1082           }
1083         }
1084       }
1085       return false;
1086     }
1087     public IsFlushWALMarker set(FlushAction... actions) {
1088       this.actions = actions;
1089       return this;
1090     }
1091   }
1092 
1093   @Test (timeout=60000)
1094   public void testFlushMarkersWALFail() throws Exception {
1095     // test the cases where the WAL append for flush markers fail.
1096     String method = name.getMethodName();
1097     TableName tableName = TableName.valueOf(method);
1098     byte[] family = Bytes.toBytes("family");
1099 
1100     // spy an actual WAL implementation to throw exception (was not able to mock)
1101     Path logDir = TEST_UTIL.getDataTestDirOnTestFS(method + "log");
1102 
1103     final Configuration walConf = new Configuration(TEST_UTIL.getConfiguration());
1104     FSUtils.setRootDir(walConf, logDir);
1105 
1106     // Make up a WAL that we can manipulate at append time.
1107     class FailAppendFlushMarkerWAL extends FSHLog {
1108       volatile FlushAction [] flushActions = null;
1109 
1110       public FailAppendFlushMarkerWAL(FileSystem fs, Path root, String logDir, Configuration conf)
1111       throws IOException {
1112         super(fs, root, logDir, conf);
1113       }
1114 
1115       @Override
1116       protected Writer createWriterInstance(Path path) throws IOException {
1117         final Writer w = super.createWriterInstance(path);
1118         return new Writer() {
1119           @Override
1120           public void close() throws IOException {
1121             w.close();
1122           }
1123 
1124           @Override
1125           public void sync(boolean forceSync) throws IOException {
1126             w.sync(forceSync);
1127           }
1128 
1129           @Override
1130           public void append(Entry entry) throws IOException {
1131             List<Cell> cells = entry.getEdit().getCells();
1132             if (WALEdit.isMetaEditFamily(cells.get(0))) {
1133                FlushDescriptor desc = WALEdit.getFlushDescriptor(cells.get(0));
1134               if (desc != null) {
1135                 for (FlushAction flushAction: flushActions) {
1136                   if (desc.getAction().equals(flushAction)) {
1137                     throw new IOException("Failed to append flush marker! " + flushAction);
1138                   }
1139                 }
1140               }
1141             }
1142             w.append(entry);
1143           }
1144 
1145           @Override
1146           public long getLength() throws IOException {
1147             return w.getLength();
1148           }
1149         };
1150       }
1151     }
1152     FailAppendFlushMarkerWAL wal =
1153       new FailAppendFlushMarkerWAL(FileSystem.get(walConf), FSUtils.getRootDir(walConf),
1154         getName(), walConf);
1155     this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW,
1156       HConstants.EMPTY_END_ROW, method, CONF, false, Durability.USE_DEFAULT, wal, family);
1157     int i = 0;
1158     Put put = new Put(Bytes.toBytes(i));
1159     put.setDurability(Durability.SKIP_WAL); // have to skip mocked wal
1160     put.add(family, Bytes.toBytes(i), Bytes.toBytes(i));
1161     region.put(put);
1162 
1163     // 1. Test case where START_FLUSH throws exception
1164     wal.flushActions = new FlushAction [] {FlushAction.START_FLUSH};
1165 
1166     // start cache flush will throw exception
1167     try {
1168       region.flush(true);
1169       fail("This should have thrown exception");
1170     } catch (DroppedSnapshotException unexpected) {
1171       // this should not be a dropped snapshot exception. Meaning that RS will not abort
1172       throw unexpected;
1173     } catch (IOException expected) {
1174       // expected
1175     }
1176     // The WAL is hosed now. It has two edits appended. We cannot roll the log without it
1177     // throwing a DroppedSnapshotException to force an abort. Just clean up the mess.
1178     region.close(true);
1179     wal.close();
1180 
1181     // 2. Test case where START_FLUSH succeeds but COMMIT_FLUSH will throw exception
1182     wal.flushActions = new FlushAction [] {FlushAction.COMMIT_FLUSH};
1183     wal = new FailAppendFlushMarkerWAL(FileSystem.get(walConf), FSUtils.getRootDir(walConf),
1184           getName(), walConf);
1185     this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW,
1186       HConstants.EMPTY_END_ROW, method, CONF, false, Durability.USE_DEFAULT, wal, family);
1187     region.put(put);
1188 
1189     // 3. Test case where ABORT_FLUSH will throw exception.
1190     // Even if ABORT_FLUSH throws exception, we should not fail with IOE, but continue with
1191     // DroppedSnapshotException. Below COMMMIT_FLUSH will cause flush to abort
1192     wal.flushActions = new FlushAction [] {FlushAction.COMMIT_FLUSH, FlushAction.ABORT_FLUSH};
1193 
1194     try {
1195       region.flush(true);
1196       fail("This should have thrown exception");
1197     } catch (DroppedSnapshotException expected) {
1198       // we expect this exception, since we were able to write the snapshot, but failed to
1199       // write the flush marker to WAL
1200     } catch (IOException unexpected) {
1201       throw unexpected;
1202     }
1203   }
1204 
1205   @Test
1206   public void testGetWhileRegionClose() throws IOException {
1207     TableName tableName = TableName.valueOf(name.getMethodName());
1208     Configuration hc = initSplit();
1209     int numRows = 100;
1210     byte[][] families = { fam1, fam2, fam3 };
1211 
1212     // Setting up region
1213     String method = name.getMethodName();
1214     this.region = initHRegion(tableName, method, hc, families);
1215     // Put data in region
1216     final int startRow = 100;
1217     putData(startRow, numRows, qual1, families);
1218     putData(startRow, numRows, qual2, families);
1219     putData(startRow, numRows, qual3, families);
1220     final AtomicBoolean done = new AtomicBoolean(false);
1221     final AtomicInteger gets = new AtomicInteger(0);
1222     GetTillDoneOrException[] threads = new GetTillDoneOrException[10];
1223     try {
1224       // Set ten threads running concurrently getting from the region.
1225       for (int i = 0; i < threads.length / 2; i++) {
1226         threads[i] = new GetTillDoneOrException(i, Bytes.toBytes("" + startRow), done, gets);
1227         threads[i].setDaemon(true);
1228         threads[i].start();
1229       }
1230       // Artificially make the condition by setting closing flag explicitly.
1231       // I can't make the issue happen with a call to region.close().
1232       this.region.closing.set(true);
1233       for (int i = threads.length / 2; i < threads.length; i++) {
1234         threads[i] = new GetTillDoneOrException(i, Bytes.toBytes("" + startRow), done, gets);
1235         threads[i].setDaemon(true);
1236         threads[i].start();
1237       }
1238     } finally {
1239       if (this.region != null) {
1240         HBaseTestingUtility.closeRegionAndWAL(this.region);
1241         this.region = null;
1242       }
1243     }
1244     done.set(true);
1245     for (GetTillDoneOrException t : threads) {
1246       try {
1247         t.join();
1248       } catch (InterruptedException e) {
1249         e.printStackTrace();
1250       }
1251       if (t.e != null) {
1252         LOG.info("Exception=" + t.e);
1253         assertFalse("Found a NPE in " + t.getName(), t.e instanceof NullPointerException);
1254       }
1255     }
1256   }
1257 
1258   /*
1259    * Thread that does get on single row until 'done' flag is flipped. If an
1260    * exception causes us to fail, it records it.
1261    */
1262   class GetTillDoneOrException extends Thread {
1263     private final Get g;
1264     private final AtomicBoolean done;
1265     private final AtomicInteger count;
1266     private Exception e;
1267 
1268     GetTillDoneOrException(final int i, final byte[] r, final AtomicBoolean d, final AtomicInteger c) {
1269       super("getter." + i);
1270       this.g = new Get(r);
1271       this.done = d;
1272       this.count = c;
1273     }
1274 
1275     @Override
1276     public void run() {
1277       while (!this.done.get()) {
1278         try {
1279           assertTrue(region.get(g).size() > 0);
1280           this.count.incrementAndGet();
1281         } catch (Exception e) {
1282           this.e = e;
1283           break;
1284         }
1285       }
1286     }
1287   }
1288 
1289   /*
1290    * An involved filter test. Has multiple column families and deletes in mix.
1291    */
1292   @Test
1293   public void testWeirdCacheBehaviour() throws Exception {
1294     byte[] TABLE = Bytes.toBytes("testWeirdCacheBehaviour");
1295     byte[][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"), Bytes.toBytes("trans-type"),
1296         Bytes.toBytes("trans-date"), Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") };
1297     this.region = initHRegion(TABLE, getName(), CONF, FAMILIES);
1298     String value = "this is the value";
1299     String value2 = "this is some other value";
1300     String keyPrefix1 = "prefix1";
1301     String keyPrefix2 = "prefix2";
1302     String keyPrefix3 = "prefix3";
1303     putRows(this.region, 3, value, keyPrefix1);
1304     putRows(this.region, 3, value, keyPrefix2);
1305     putRows(this.region, 3, value, keyPrefix3);
1306     putRows(this.region, 3, value2, keyPrefix1);
1307     putRows(this.region, 3, value2, keyPrefix2);
1308     putRows(this.region, 3, value2, keyPrefix3);
1309     System.out.println("Checking values for key: " + keyPrefix1);
1310     assertEquals("Got back incorrect number of rows from scan", 3,
1311         getNumberOfRows(keyPrefix1, value2, this.region));
1312     System.out.println("Checking values for key: " + keyPrefix2);
1313     assertEquals("Got back incorrect number of rows from scan", 3,
1314         getNumberOfRows(keyPrefix2, value2, this.region));
1315     System.out.println("Checking values for key: " + keyPrefix3);
1316     assertEquals("Got back incorrect number of rows from scan", 3,
1317         getNumberOfRows(keyPrefix3, value2, this.region));
1318     deleteColumns(this.region, value2, keyPrefix1);
1319     deleteColumns(this.region, value2, keyPrefix2);
1320     deleteColumns(this.region, value2, keyPrefix3);
1321     System.out.println("Starting important checks.....");
1322     assertEquals("Got back incorrect number of rows from scan: " + keyPrefix1, 0,
1323         getNumberOfRows(keyPrefix1, value2, this.region));
1324     assertEquals("Got back incorrect number of rows from scan: " + keyPrefix2, 0,
1325         getNumberOfRows(keyPrefix2, value2, this.region));
1326     assertEquals("Got back incorrect number of rows from scan: " + keyPrefix3, 0,
1327         getNumberOfRows(keyPrefix3, value2, this.region));
1328   }
1329 
1330   @Test
1331   public void testAppendWithReadOnlyTable() throws Exception {
1332     byte[] TABLE = Bytes.toBytes("readOnlyTable");
1333     this.region = initHRegion(TABLE, getName(), CONF, true, Bytes.toBytes("somefamily"));
1334     boolean exceptionCaught = false;
1335     Append append = new Append(Bytes.toBytes("somerow"));
1336     append.setDurability(Durability.SKIP_WAL);
1337     append.add(Bytes.toBytes("somefamily"), Bytes.toBytes("somequalifier"),
1338         Bytes.toBytes("somevalue"));
1339     try {
1340       region.append(append);
1341     } catch (IOException e) {
1342       exceptionCaught = true;
1343     }
1344     assertTrue(exceptionCaught);
1345   }
1346 
1347   @Test
1348   public void testIncrWithReadOnlyTable() throws Exception {
1349     byte[] TABLE = Bytes.toBytes("readOnlyTable");
1350     this.region = initHRegion(TABLE, getName(), CONF, true, Bytes.toBytes("somefamily"));
1351     boolean exceptionCaught = false;
1352     Increment inc = new Increment(Bytes.toBytes("somerow"));
1353     inc.setDurability(Durability.SKIP_WAL);
1354     inc.addColumn(Bytes.toBytes("somefamily"), Bytes.toBytes("somequalifier"), 1L);
1355     try {
1356       region.increment(inc);
1357     } catch (IOException e) {
1358       exceptionCaught = true;
1359     }
1360     assertTrue(exceptionCaught);
1361   }
1362 
1363   private void deleteColumns(HRegion r, String value, String keyPrefix) throws IOException {
1364     InternalScanner scanner = buildScanner(keyPrefix, value, r);
1365     int count = 0;
1366     boolean more = false;
1367     List<Cell> results = new ArrayList<Cell>();
1368     do {
1369       more = scanner.next(results);
1370       if (results != null && !results.isEmpty())
1371         count++;
1372       else
1373         break;
1374       Delete delete = new Delete(CellUtil.cloneRow(results.get(0)));
1375       delete.deleteColumn(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"));
1376       r.delete(delete);
1377       results.clear();
1378     } while (more);
1379     assertEquals("Did not perform correct number of deletes", 3, count);
1380   }
1381 
1382   private int getNumberOfRows(String keyPrefix, String value, HRegion r) throws Exception {
1383     InternalScanner resultScanner = buildScanner(keyPrefix, value, r);
1384     int numberOfResults = 0;
1385     List<Cell> results = new ArrayList<Cell>();
1386     boolean more = false;
1387     do {
1388       more = resultScanner.next(results);
1389       if (results != null && !results.isEmpty())
1390         numberOfResults++;
1391       else
1392         break;
1393       for (Cell kv : results) {
1394         System.out.println("kv=" + kv.toString() + ", " + Bytes.toString(CellUtil.cloneValue(kv)));
1395       }
1396       results.clear();
1397     } while (more);
1398     return numberOfResults;
1399   }
1400 
1401   private InternalScanner buildScanner(String keyPrefix, String value, HRegion r)
1402       throws IOException {
1403     // Defaults FilterList.Operator.MUST_PASS_ALL.
1404     FilterList allFilters = new FilterList();
1405     allFilters.addFilter(new PrefixFilter(Bytes.toBytes(keyPrefix)));
1406     // Only return rows where this column value exists in the row.
1407     SingleColumnValueFilter filter = new SingleColumnValueFilter(Bytes.toBytes("trans-tags"),
1408         Bytes.toBytes("qual2"), CompareOp.EQUAL, Bytes.toBytes(value));
1409     filter.setFilterIfMissing(true);
1410     allFilters.addFilter(filter);
1411     Scan scan = new Scan();
1412     scan.addFamily(Bytes.toBytes("trans-blob"));
1413     scan.addFamily(Bytes.toBytes("trans-type"));
1414     scan.addFamily(Bytes.toBytes("trans-date"));
1415     scan.addFamily(Bytes.toBytes("trans-tags"));
1416     scan.addFamily(Bytes.toBytes("trans-group"));
1417     scan.setFilter(allFilters);
1418     return r.getScanner(scan);
1419   }
1420 
1421   private void putRows(HRegion r, int numRows, String value, String key) throws IOException {
1422     for (int i = 0; i < numRows; i++) {
1423       String row = key + "_" + i/* UUID.randomUUID().toString() */;
1424       System.out.println(String.format("Saving row: %s, with value %s", row, value));
1425       Put put = new Put(Bytes.toBytes(row));
1426       put.setDurability(Durability.SKIP_WAL);
1427       put.add(Bytes.toBytes("trans-blob"), null, Bytes.toBytes("value for blob"));
1428       put.add(Bytes.toBytes("trans-type"), null, Bytes.toBytes("statement"));
1429       put.add(Bytes.toBytes("trans-date"), null, Bytes.toBytes("20090921010101999"));
1430       put.add(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"), Bytes.toBytes(value));
1431       put.add(Bytes.toBytes("trans-group"), null, Bytes.toBytes("adhocTransactionGroupId"));
1432       r.put(put);
1433     }
1434   }
1435 
1436   @Test
1437   public void testFamilyWithAndWithoutColon() throws Exception {
1438     byte[] b = Bytes.toBytes(getName());
1439     byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
1440     this.region = initHRegion(b, getName(), CONF, cf);
1441     Put p = new Put(b);
1442     byte[] cfwithcolon = Bytes.toBytes(COLUMN_FAMILY + ":");
1443     p.add(cfwithcolon, cfwithcolon, cfwithcolon);
1444     boolean exception = false;
1445     try {
1446       this.region.put(p);
1447     } catch (NoSuchColumnFamilyException e) {
1448       exception = true;
1449     }
1450     assertTrue(exception);
1451   }
1452 
1453   @Test
1454   public void testBatchPut_whileNoRowLocksHeld() throws IOException {
1455     byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
1456     byte[] qual = Bytes.toBytes("qual");
1457     byte[] val = Bytes.toBytes("val");
1458     this.region = initHRegion(Bytes.toBytes(getName()), getName(), CONF, cf);
1459     MetricsWALSource source = CompatibilitySingletonFactory.getInstance(MetricsWALSource.class);
1460     long syncs = metricsAssertHelper.getCounter("syncTimeNumOps", source);
1461     metricsAssertHelper.assertCounter("syncTimeNumOps", syncs, source);
1462 
1463     LOG.info("First a batch put with all valid puts");
1464     final Put[] puts = new Put[10];
1465     for (int i = 0; i < 10; i++) {
1466       puts[i] = new Put(Bytes.toBytes("row_" + i));
1467       puts[i].add(cf, qual, val);
1468     }
1469 
1470     OperationStatus[] codes = this.region.batchMutate(puts);
1471     assertEquals(10, codes.length);
1472     for (int i = 0; i < 10; i++) {
1473       assertEquals(OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
1474     }
1475     metricsAssertHelper.assertCounter("syncTimeNumOps", syncs + 1, source);
1476 
1477     LOG.info("Next a batch put with one invalid family");
1478     puts[5].add(Bytes.toBytes("BAD_CF"), qual, val);
1479     codes = this.region.batchMutate(puts);
1480     assertEquals(10, codes.length);
1481     for (int i = 0; i < 10; i++) {
1482       assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY : OperationStatusCode.SUCCESS,
1483           codes[i].getOperationStatusCode());
1484     }
1485 
1486     metricsAssertHelper.assertCounter("syncTimeNumOps", syncs + 2, source);
1487   }
1488 
1489   @Test
1490   public void testBatchPut_whileMultipleRowLocksHeld() throws Exception {
1491     byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
1492     byte[] qual = Bytes.toBytes("qual");
1493     byte[] val = Bytes.toBytes("val");
1494     this.region = initHRegion(Bytes.toBytes(getName()), getName(), CONF, cf);
1495     MetricsWALSource source = CompatibilitySingletonFactory.getInstance(MetricsWALSource.class);
1496     long syncs = metricsAssertHelper.getCounter("syncTimeNumOps", source);
1497     metricsAssertHelper.assertCounter("syncTimeNumOps", syncs, source);
1498 
1499     final Put[] puts = new Put[10];
1500     for (int i = 0; i < 10; i++) {
1501       puts[i] = new Put(Bytes.toBytes("row_" + i));
1502       puts[i].add(cf, qual, val);
1503     }
1504     puts[5].add(Bytes.toBytes("BAD_CF"), qual, val);
1505 
1506     LOG.info("batchPut will have to break into four batches to avoid row locks");
1507     RowLock rowLock1 = region.getRowLock(Bytes.toBytes("row_2"));
1508     RowLock rowLock2 = region.getRowLock(Bytes.toBytes("row_1"));
1509     RowLock rowLock3 = region.getRowLock(Bytes.toBytes("row_3"));
1510     RowLock rowLock4 = region.getRowLock(Bytes.toBytes("row_3"), true);
1511 
1512 
1513     MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext(CONF);
1514     final AtomicReference<OperationStatus[]> retFromThread = new AtomicReference<>();
1515     final CountDownLatch startingPuts = new CountDownLatch(1);
1516     final CountDownLatch startingClose = new CountDownLatch(1);
1517     TestThread putter = new TestThread(ctx) {
1518       @Override
1519       public void doWork() throws IOException {
1520         startingPuts.countDown();
1521         retFromThread.set(region.batchMutate(puts));
1522       }
1523     };
1524     LOG.info("...starting put thread while holding locks");
1525     ctx.addThread(putter);
1526     ctx.startThreads();
1527 
1528     // Now attempt to close the region from another thread.  Prior to HBASE-12565
1529     // this would cause the in-progress batchMutate operation to to fail with
1530     // exception because it use to release and re-acquire the close-guard lock
1531     // between batches.  Caller then didn't get status indicating which writes succeeded.
1532     // We now expect this thread to block until the batchMutate call finishes.
1533     Thread regionCloseThread = new TestThread(ctx) {
1534       @Override
1535       public void doWork() {
1536         try {
1537           startingPuts.await();
1538           // Give some time for the batch mutate to get in.
1539           // We don't want to race with the mutate
1540           Thread.sleep(10);
1541           startingClose.countDown();
1542           HBaseTestingUtility.closeRegionAndWAL(region);
1543           region = null;
1544         } catch (IOException e) {
1545           throw new RuntimeException(e);
1546         } catch (InterruptedException e) {
1547           throw new RuntimeException(e);
1548         }
1549       }
1550     };
1551     regionCloseThread.start();
1552 
1553     startingClose.await();
1554     startingPuts.await();
1555     Thread.sleep(100);
1556     LOG.info("...releasing row lock 1, which should let put thread continue");
1557     rowLock1.release();
1558     rowLock2.release();
1559     rowLock3.release();
1560     waitForCounter(source, "syncTimeNumOps", syncs + 1);
1561 
1562     LOG.info("...joining on put thread");
1563     ctx.stop();
1564     regionCloseThread.join();
1565 
1566     OperationStatus[] codes = retFromThread.get();
1567     for (int i = 0; i < codes.length; i++) {
1568       assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY : OperationStatusCode.SUCCESS,
1569           codes[i].getOperationStatusCode());
1570     }
1571     rowLock4.release();
1572   }
1573 
1574   private void waitForCounter(MetricsWALSource source, String metricName, long expectedCount)
1575       throws InterruptedException {
1576     long startWait = System.currentTimeMillis();
1577     long currentCount;
1578     while ((currentCount = metricsAssertHelper.getCounter(metricName, source)) < expectedCount) {
1579       Thread.sleep(100);
1580       if (System.currentTimeMillis() - startWait > 10000) {
1581         fail(String.format("Timed out waiting for '%s' >= '%s', currentCount=%s", metricName,
1582           expectedCount, currentCount));
1583       }
1584     }
1585   }
1586 
1587   @Test
1588   public void testBatchPutWithTsSlop() throws Exception {
1589     byte[] b = Bytes.toBytes(getName());
1590     byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
1591     byte[] qual = Bytes.toBytes("qual");
1592     byte[] val = Bytes.toBytes("val");
1593 
1594     // add data with a timestamp that is too recent for range. Ensure assert
1595     CONF.setInt("hbase.hregion.keyvalue.timestamp.slop.millisecs", 1000);
1596     this.region = initHRegion(b, getName(), CONF, cf);
1597 
1598     MetricsWALSource source = CompatibilitySingletonFactory.getInstance(MetricsWALSource.class);
1599     long syncs = metricsAssertHelper.getCounter("syncTimeNumOps", source);
1600     metricsAssertHelper.assertCounter("syncTimeNumOps", syncs, source);
1601 
1602     final Put[] puts = new Put[10];
1603     for (int i = 0; i < 10; i++) {
1604       puts[i] = new Put(Bytes.toBytes("row_" + i), Long.MAX_VALUE - 100);
1605       puts[i].add(cf, qual, val);
1606     }
1607 
1608     OperationStatus[] codes = this.region.batchMutate(puts);
1609     assertEquals(10, codes.length);
1610     for (int i = 0; i < 10; i++) {
1611       assertEquals(OperationStatusCode.SANITY_CHECK_FAILURE, codes[i].getOperationStatusCode());
1612     }
1613     metricsAssertHelper.assertCounter("syncTimeNumOps", syncs, source);
1614   }
1615 
1616   // ////////////////////////////////////////////////////////////////////////////
1617   // checkAndMutate tests
1618   // ////////////////////////////////////////////////////////////////////////////
1619   @Test
1620   public void testCheckAndMutate_WithEmptyRowValue() throws IOException {
1621     byte[] row1 = Bytes.toBytes("row1");
1622     byte[] fam1 = Bytes.toBytes("fam1");
1623     byte[] qf1 = Bytes.toBytes("qualifier");
1624     byte[] emptyVal = new byte[] {};
1625     byte[] val1 = Bytes.toBytes("value1");
1626     byte[] val2 = Bytes.toBytes("value2");
1627 
1628     // Setting up region
1629     String method = this.getName();
1630     this.region = initHRegion(tableName, method, CONF, fam1);
1631     // Putting empty data in key
1632     Put put = new Put(row1);
1633     put.add(fam1, qf1, emptyVal);
1634 
1635     // checkAndPut with empty value
1636     boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(
1637         emptyVal), put, true);
1638     assertTrue(res);
1639 
1640     // Putting data in key
1641     put = new Put(row1);
1642     put.add(fam1, qf1, val1);
1643 
1644     // checkAndPut with correct value
1645     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(emptyVal),
1646         put, true);
1647     assertTrue(res);
1648 
1649     // not empty anymore
1650     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(emptyVal),
1651         put, true);
1652     assertFalse(res);
1653 
1654     Delete delete = new Delete(row1);
1655     delete.deleteColumn(fam1, qf1);
1656     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(emptyVal),
1657         delete, true);
1658     assertFalse(res);
1659 
1660     put = new Put(row1);
1661     put.add(fam1, qf1, val2);
1662     // checkAndPut with correct value
1663     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(val1),
1664         put, true);
1665     assertTrue(res);
1666 
1667     // checkAndDelete with correct value
1668     delete = new Delete(row1);
1669     delete.deleteColumn(fam1, qf1);
1670     delete.deleteColumn(fam1, qf1);
1671     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(val2),
1672         delete, true);
1673     assertTrue(res);
1674 
1675     delete = new Delete(row1);
1676     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(emptyVal),
1677         delete, true);
1678     assertTrue(res);
1679 
1680     // checkAndPut looking for a null value
1681     put = new Put(row1);
1682     put.add(fam1, qf1, val1);
1683 
1684     res = region
1685         .checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new NullComparator(), put, true);
1686     assertTrue(res);
1687   }
1688 
1689   @Test
1690   public void testCheckAndMutate_WithWrongValue() throws IOException {
1691     byte[] row1 = Bytes.toBytes("row1");
1692     byte[] fam1 = Bytes.toBytes("fam1");
1693     byte[] qf1 = Bytes.toBytes("qualifier");
1694     byte[] val1 = Bytes.toBytes("value1");
1695     byte[] val2 = Bytes.toBytes("value2");
1696 
1697     // Setting up region
1698     String method = this.getName();
1699     this.region = initHRegion(tableName, method, CONF, fam1);
1700     // Putting data in key
1701     Put put = new Put(row1);
1702     put.add(fam1, qf1, val1);
1703     region.put(put);
1704 
1705     // checkAndPut with wrong value
1706     boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(
1707         val2), put, true);
1708     assertFalse(res);
1709 
1710     // checkAndDelete with wrong value
1711     Delete delete = new Delete(row1);
1712     delete.deleteFamily(fam1);
1713     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(val2),
1714         put, true);
1715     assertFalse(res);
1716   }
1717 
1718   @Test
1719   public void testCheckAndMutate_WithCorrectValue() throws IOException {
1720     byte[] row1 = Bytes.toBytes("row1");
1721     byte[] fam1 = Bytes.toBytes("fam1");
1722     byte[] qf1 = Bytes.toBytes("qualifier");
1723     byte[] val1 = Bytes.toBytes("value1");
1724 
1725     // Setting up region
1726     String method = this.getName();
1727     this.region = initHRegion(tableName, method, CONF, fam1);
1728     // Putting data in key
1729     Put put = new Put(row1);
1730     put.add(fam1, qf1, val1);
1731     region.put(put);
1732 
1733     // checkAndPut with correct value
1734     boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(
1735         val1), put, true);
1736     assertTrue( res);
1737 
1738     // checkAndDelete with correct value
1739     Delete delete = new Delete(row1);
1740     delete.deleteColumn(fam1, qf1);
1741     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(val1),
1742         delete, true);
1743     assertTrue(res);
1744   }
1745 
1746   @Test
1747   public void testCheckAndMutate_WithNonEqualCompareOp() throws IOException {
1748     byte[] row1 = Bytes.toBytes("row1");
1749     byte[] fam1 = Bytes.toBytes("fam1");
1750     byte[] qf1 = Bytes.toBytes("qualifier");
1751     byte[] val1 = Bytes.toBytes("value1");
1752     byte[] val2 = Bytes.toBytes("value2");
1753     byte[] val3 = Bytes.toBytes("value3");
1754     byte[] val4 = Bytes.toBytes("value4");
1755 
1756     // Setting up region
1757     String method = this.getName();
1758     this.region = initHRegion(tableName, method, CONF, fam1);
1759     // Putting val3 in key
1760     Put put = new Put(row1);
1761     put.add(fam1, qf1, val3);
1762     region.put(put);
1763 
1764     // Test CompareOp.LESS: original = val3, compare with val3, fail
1765     boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS,
1766         new BinaryComparator(val3), put, true);
1767     assertEquals(false, res);
1768 
1769     // Test CompareOp.LESS: original = val3, compare with val4, fail
1770     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS,
1771         new BinaryComparator(val4), put, true);
1772     assertEquals(false, res);
1773 
1774     // Test CompareOp.LESS: original = val3, compare with val2,
1775     // succeed (now value = val2)
1776     put = new Put(row1);
1777     put.add(fam1, qf1, val2);
1778     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS,
1779         new BinaryComparator(val2), put, true);
1780     assertEquals(true, res);
1781 
1782     // Test CompareOp.LESS_OR_EQUAL: original = val2, compare with val3, fail
1783     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS_OR_EQUAL,
1784         new BinaryComparator(val3), put, true);
1785     assertEquals(false, res);
1786 
1787     // Test CompareOp.LESS_OR_EQUAL: original = val2, compare with val2,
1788     // succeed (value still = val2)
1789     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS_OR_EQUAL,
1790         new BinaryComparator(val2), put, true);
1791     assertEquals(true, res);
1792 
1793     // Test CompareOp.LESS_OR_EQUAL: original = val2, compare with val1,
1794     // succeed (now value = val3)
1795     put = new Put(row1);
1796     put.add(fam1, qf1, val3);
1797     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS_OR_EQUAL,
1798         new BinaryComparator(val1), put, true);
1799     assertEquals(true, res);
1800 
1801     // Test CompareOp.GREATER: original = val3, compare with val3, fail
1802     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER,
1803         new BinaryComparator(val3), put, true);
1804     assertEquals(false, res);
1805 
1806     // Test CompareOp.GREATER: original = val3, compare with val2, fail
1807     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER,
1808         new BinaryComparator(val2), put, true);
1809     assertEquals(false, res);
1810 
1811     // Test CompareOp.GREATER: original = val3, compare with val4,
1812     // succeed (now value = val2)
1813     put = new Put(row1);
1814     put.add(fam1, qf1, val2);
1815     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER,
1816         new BinaryComparator(val4), put, true);
1817     assertEquals(true, res);
1818 
1819     // Test CompareOp.GREATER_OR_EQUAL: original = val2, compare with val1, fail
1820     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER_OR_EQUAL,
1821         new BinaryComparator(val1), put, true);
1822     assertEquals(false, res);
1823 
1824     // Test CompareOp.GREATER_OR_EQUAL: original = val2, compare with val2,
1825     // succeed (value still = val2)
1826     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER_OR_EQUAL,
1827         new BinaryComparator(val2), put, true);
1828     assertEquals(true, res);
1829 
1830     // Test CompareOp.GREATER_OR_EQUAL: original = val2, compare with val3, succeed
1831     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER_OR_EQUAL,
1832         new BinaryComparator(val3), put, true);
1833     assertEquals(true, res);
1834   }
1835 
1836   @Test
1837   public void testCheckAndPut_ThatPutWasWritten() throws IOException {
1838     byte[] row1 = Bytes.toBytes("row1");
1839     byte[] fam1 = Bytes.toBytes("fam1");
1840     byte[] fam2 = Bytes.toBytes("fam2");
1841     byte[] qf1 = Bytes.toBytes("qualifier");
1842     byte[] val1 = Bytes.toBytes("value1");
1843     byte[] val2 = Bytes.toBytes("value2");
1844 
1845     byte[][] families = { fam1, fam2 };
1846 
1847     // Setting up region
1848     String method = this.getName();
1849     this.region = initHRegion(tableName, method, CONF, families);
1850     // Putting data in the key to check
1851     Put put = new Put(row1);
1852     put.add(fam1, qf1, val1);
1853     region.put(put);
1854 
1855     // Creating put to add
1856     long ts = System.currentTimeMillis();
1857     KeyValue kv = new KeyValue(row1, fam2, qf1, ts, KeyValue.Type.Put, val2);
1858     put = new Put(row1);
1859     put.add(kv);
1860 
1861     // checkAndPut with wrong value
1862     boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(
1863         val1), put, true);
1864     assertEquals(true, res);
1865 
1866     Get get = new Get(row1);
1867     get.addColumn(fam2, qf1);
1868     Cell[] actual = region.get(get).rawCells();
1869 
1870     Cell[] expected = { kv };
1871 
1872     assertEquals(expected.length, actual.length);
1873     for (int i = 0; i < actual.length; i++) {
1874       assertEquals(expected[i], actual[i]);
1875     }
1876   }
1877 
1878   @Test
1879   public void testCheckAndPut_wrongRowInPut() throws IOException {
1880     TableName tableName = TableName.valueOf(name.getMethodName());
1881     this.region = initHRegion(tableName, this.getName(), CONF, COLUMNS);
1882     Put put = new Put(row2);
1883     put.add(fam1, qual1, value1);
1884     try {
1885       region.checkAndMutate(row, fam1, qual1, CompareOp.EQUAL,
1886           new BinaryComparator(value2), put, false);
1887       fail();
1888     } catch (org.apache.hadoop.hbase.DoNotRetryIOException expected) {
1889       // expected exception.
1890     }
1891   }
1892 
1893   @Test
1894   public void testCheckAndDelete_ThatDeleteWasWritten() throws IOException {
1895     byte[] row1 = Bytes.toBytes("row1");
1896     byte[] fam1 = Bytes.toBytes("fam1");
1897     byte[] fam2 = Bytes.toBytes("fam2");
1898     byte[] qf1 = Bytes.toBytes("qualifier1");
1899     byte[] qf2 = Bytes.toBytes("qualifier2");
1900     byte[] qf3 = Bytes.toBytes("qualifier3");
1901     byte[] val1 = Bytes.toBytes("value1");
1902     byte[] val2 = Bytes.toBytes("value2");
1903     byte[] val3 = Bytes.toBytes("value3");
1904     byte[] emptyVal = new byte[] {};
1905 
1906     byte[][] families = { fam1, fam2 };
1907 
1908     // Setting up region
1909     String method = this.getName();
1910     this.region = initHRegion(tableName, method, CONF, families);
1911     // Put content
1912     Put put = new Put(row1);
1913     put.add(fam1, qf1, val1);
1914     region.put(put);
1915     Threads.sleep(2);
1916 
1917     put = new Put(row1);
1918     put.add(fam1, qf1, val2);
1919     put.add(fam2, qf1, val3);
1920     put.add(fam2, qf2, val2);
1921     put.add(fam2, qf3, val1);
1922     put.add(fam1, qf3, val1);
1923     region.put(put);
1924 
1925     // Multi-column delete
1926     Delete delete = new Delete(row1);
1927     delete.deleteColumn(fam1, qf1);
1928     delete.deleteColumn(fam2, qf1);
1929     delete.deleteColumn(fam1, qf3);
1930     boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(
1931         val2), delete, true);
1932     assertEquals(true, res);
1933 
1934     Get get = new Get(row1);
1935     get.addColumn(fam1, qf1);
1936     get.addColumn(fam1, qf3);
1937     get.addColumn(fam2, qf2);
1938     Result r = region.get(get);
1939     assertEquals(2, r.size());
1940     assertArrayEquals(val1, r.getValue(fam1, qf1));
1941     assertArrayEquals(val2, r.getValue(fam2, qf2));
1942 
1943     // Family delete
1944     delete = new Delete(row1);
1945     delete.deleteFamily(fam2);
1946     res = region.checkAndMutate(row1, fam2, qf1, CompareOp.EQUAL, new BinaryComparator(emptyVal),
1947         delete, true);
1948     assertEquals(true, res);
1949 
1950     get = new Get(row1);
1951     r = region.get(get);
1952     assertEquals(1, r.size());
1953     assertArrayEquals(val1, r.getValue(fam1, qf1));
1954 
1955     // Row delete
1956     delete = new Delete(row1);
1957     res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(val1),
1958         delete, true);
1959     assertEquals(true, res);
1960     get = new Get(row1);
1961     r = region.get(get);
1962     assertEquals(0, r.size());
1963   }
1964 
1965   // ////////////////////////////////////////////////////////////////////////////
1966   // Delete tests
1967   // ////////////////////////////////////////////////////////////////////////////
1968   @Test
1969   public void testDelete_multiDeleteColumn() throws IOException {
1970     byte[] row1 = Bytes.toBytes("row1");
1971     byte[] fam1 = Bytes.toBytes("fam1");
1972     byte[] qual = Bytes.toBytes("qualifier");
1973     byte[] value = Bytes.toBytes("value");
1974 
1975     Put put = new Put(row1);
1976     put.add(fam1, qual, 1, value);
1977     put.add(fam1, qual, 2, value);
1978 
1979     String method = this.getName();
1980     this.region = initHRegion(tableName, method, CONF, fam1);
1981     region.put(put);
1982 
1983     // We do support deleting more than 1 'latest' version
1984     Delete delete = new Delete(row1);
1985     delete.deleteColumn(fam1, qual);
1986     delete.deleteColumn(fam1, qual);
1987     region.delete(delete);
1988 
1989     Get get = new Get(row1);
1990     get.addFamily(fam1);
1991     Result r = region.get(get);
1992     assertEquals(0, r.size());
1993   }
1994 
1995   @Test
1996   public void testDelete_CheckFamily() throws IOException {
1997     byte[] row1 = Bytes.toBytes("row1");
1998     byte[] fam1 = Bytes.toBytes("fam1");
1999     byte[] fam2 = Bytes.toBytes("fam2");
2000     byte[] fam3 = Bytes.toBytes("fam3");
2001     byte[] fam4 = Bytes.toBytes("fam4");
2002 
2003     // Setting up region
2004     String method = this.getName();
2005     this.region = initHRegion(tableName, method, CONF, fam1, fam2, fam3);
2006     List<Cell> kvs = new ArrayList<Cell>();
2007     kvs.add(new KeyValue(row1, fam4, null, null));
2008 
2009     // testing existing family
2010     byte[] family = fam2;
2011     try {
2012       NavigableMap<byte[], List<Cell>> deleteMap = new TreeMap<byte[], List<Cell>>(
2013           Bytes.BYTES_COMPARATOR);
2014       deleteMap.put(family, kvs);
2015       region.delete(deleteMap, Durability.SYNC_WAL);
2016     } catch (Exception e) {
2017       assertTrue("Family " + new String(family) + " does not exist", false);
2018     }
2019 
2020     // testing non existing family
2021     boolean ok = false;
2022     family = fam4;
2023     try {
2024       NavigableMap<byte[], List<Cell>> deleteMap = new TreeMap<byte[], List<Cell>>(
2025           Bytes.BYTES_COMPARATOR);
2026       deleteMap.put(family, kvs);
2027       region.delete(deleteMap, Durability.SYNC_WAL);
2028     } catch (Exception e) {
2029       ok = true;
2030     }
2031     assertTrue("Family " + new String(family) + " does exist", ok);
2032   }
2033 
2034   @Test
2035   public void testDelete_mixed() throws IOException, InterruptedException {
2036     byte[] fam = Bytes.toBytes("info");
2037     byte[][] families = { fam };
2038     String method = this.getName();
2039     this.region = initHRegion(tableName, method, CONF, families);
2040     EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
2041 
2042     byte[] row = Bytes.toBytes("table_name");
2043     // column names
2044     byte[] serverinfo = Bytes.toBytes("serverinfo");
2045     byte[] splitA = Bytes.toBytes("splitA");
2046     byte[] splitB = Bytes.toBytes("splitB");
2047 
2048     // add some data:
2049     Put put = new Put(row);
2050     put.add(fam, splitA, Bytes.toBytes("reference_A"));
2051     region.put(put);
2052 
2053     put = new Put(row);
2054     put.add(fam, splitB, Bytes.toBytes("reference_B"));
2055     region.put(put);
2056 
2057     put = new Put(row);
2058     put.add(fam, serverinfo, Bytes.toBytes("ip_address"));
2059     region.put(put);
2060 
2061     // ok now delete a split:
2062     Delete delete = new Delete(row);
2063     delete.deleteColumns(fam, splitA);
2064     region.delete(delete);
2065 
2066     // assert some things:
2067     Get get = new Get(row).addColumn(fam, serverinfo);
2068     Result result = region.get(get);
2069     assertEquals(1, result.size());
2070 
2071     get = new Get(row).addColumn(fam, splitA);
2072     result = region.get(get);
2073     assertEquals(0, result.size());
2074 
2075     get = new Get(row).addColumn(fam, splitB);
2076     result = region.get(get);
2077     assertEquals(1, result.size());
2078 
2079     // Assert that after a delete, I can put.
2080     put = new Put(row);
2081     put.add(fam, splitA, Bytes.toBytes("reference_A"));
2082     region.put(put);
2083     get = new Get(row);
2084     result = region.get(get);
2085     assertEquals(3, result.size());
2086 
2087     // Now delete all... then test I can add stuff back
2088     delete = new Delete(row);
2089     region.delete(delete);
2090     assertEquals(0, region.get(get).size());
2091 
2092     region.put(new Put(row).add(fam, splitA, Bytes.toBytes("reference_A")));
2093     result = region.get(get);
2094     assertEquals(1, result.size());
2095   }
2096 
2097   @Test
2098   public void testDeleteRowWithFutureTs() throws IOException {
2099     byte[] fam = Bytes.toBytes("info");
2100     byte[][] families = { fam };
2101     String method = this.getName();
2102     this.region = initHRegion(tableName, method, CONF, families);
2103     byte[] row = Bytes.toBytes("table_name");
2104     // column names
2105     byte[] serverinfo = Bytes.toBytes("serverinfo");
2106 
2107     // add data in the far future
2108     Put put = new Put(row);
2109     put.add(fam, serverinfo, HConstants.LATEST_TIMESTAMP - 5, Bytes.toBytes("value"));
2110     region.put(put);
2111 
2112     // now delete something in the present
2113     Delete delete = new Delete(row);
2114     region.delete(delete);
2115 
2116     // make sure we still see our data
2117     Get get = new Get(row).addColumn(fam, serverinfo);
2118     Result result = region.get(get);
2119     assertEquals(1, result.size());
2120 
2121     // delete the future row
2122     delete = new Delete(row, HConstants.LATEST_TIMESTAMP - 3);
2123     region.delete(delete);
2124 
2125     // make sure it is gone
2126     get = new Get(row).addColumn(fam, serverinfo);
2127     result = region.get(get);
2128     assertEquals(0, result.size());
2129   }
2130 
2131   /**
2132    * Tests that the special LATEST_TIMESTAMP option for puts gets replaced by
2133    * the actual timestamp
2134    */
2135   @Test
2136   public void testPutWithLatestTS() throws IOException {
2137     byte[] fam = Bytes.toBytes("info");
2138     byte[][] families = { fam };
2139     String method = this.getName();
2140     this.region = initHRegion(tableName, method, CONF, families);
2141     byte[] row = Bytes.toBytes("row1");
2142     // column names
2143     byte[] qual = Bytes.toBytes("qual");
2144 
2145     // add data with LATEST_TIMESTAMP, put without WAL
2146     Put put = new Put(row);
2147     put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
2148     region.put(put);
2149 
2150     // Make sure it shows up with an actual timestamp
2151     Get get = new Get(row).addColumn(fam, qual);
2152     Result result = region.get(get);
2153     assertEquals(1, result.size());
2154     Cell kv = result.rawCells()[0];
2155     LOG.info("Got: " + kv);
2156     assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
2157         kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
2158 
2159     // Check same with WAL enabled (historically these took different
2160     // code paths, so check both)
2161     row = Bytes.toBytes("row2");
2162     put = new Put(row);
2163     put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
2164     region.put(put);
2165 
2166     // Make sure it shows up with an actual timestamp
2167     get = new Get(row).addColumn(fam, qual);
2168     result = region.get(get);
2169     assertEquals(1, result.size());
2170     kv = result.rawCells()[0];
2171     LOG.info("Got: " + kv);
2172     assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
2173         kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
2174   }
2175 
2176   /**
2177    * Tests that there is server-side filtering for invalid timestamp upper
2178    * bound. Note that the timestamp lower bound is automatically handled for us
2179    * by the TTL field.
2180    */
2181   @Test
2182   public void testPutWithTsSlop() throws IOException {
2183     byte[] fam = Bytes.toBytes("info");
2184     byte[][] families = { fam };
2185     String method = this.getName();
2186 
2187     // add data with a timestamp that is too recent for range. Ensure assert
2188     CONF.setInt("hbase.hregion.keyvalue.timestamp.slop.millisecs", 1000);
2189     this.region = initHRegion(tableName, method, CONF, families);
2190     boolean caughtExcep = false;
2191     try {
2192       // no TS specified == use latest. should not error
2193       region.put(new Put(row).add(fam, Bytes.toBytes("qual"), Bytes.toBytes("value")));
2194       // TS out of range. should error
2195       region.put(new Put(row).add(fam, Bytes.toBytes("qual"), System.currentTimeMillis() + 2000,
2196           Bytes.toBytes("value")));
2197       fail("Expected IOE for TS out of configured timerange");
2198     } catch (FailedSanityCheckException ioe) {
2199       LOG.debug("Received expected exception", ioe);
2200       caughtExcep = true;
2201     }
2202     assertTrue("Should catch FailedSanityCheckException", caughtExcep);
2203   }
2204 
2205   @Test
2206   public void testScanner_DeleteOneFamilyNotAnother() throws IOException {
2207     byte[] fam1 = Bytes.toBytes("columnA");
2208     byte[] fam2 = Bytes.toBytes("columnB");
2209     this.region = initHRegion(tableName, getName(), CONF, fam1, fam2);
2210     byte[] rowA = Bytes.toBytes("rowA");
2211     byte[] rowB = Bytes.toBytes("rowB");
2212 
2213     byte[] value = Bytes.toBytes("value");
2214 
2215     Delete delete = new Delete(rowA);
2216     delete.deleteFamily(fam1);
2217 
2218     region.delete(delete);
2219 
2220     // now create data.
2221     Put put = new Put(rowA);
2222     put.add(fam2, null, value);
2223     region.put(put);
2224 
2225     put = new Put(rowB);
2226     put.add(fam1, null, value);
2227     put.add(fam2, null, value);
2228     region.put(put);
2229 
2230     Scan scan = new Scan();
2231     scan.addFamily(fam1).addFamily(fam2);
2232     InternalScanner s = region.getScanner(scan);
2233     List<Cell> results = new ArrayList<Cell>();
2234     s.next(results);
2235     assertTrue(CellUtil.matchingRow(results.get(0), rowA));
2236 
2237     results.clear();
2238     s.next(results);
2239     assertTrue(CellUtil.matchingRow(results.get(0), rowB));
2240   }
2241 
2242   @Test
2243   public void testDataInMemoryWithoutWAL() throws IOException {
2244     FileSystem fs = FileSystem.get(CONF);
2245     Path rootDir = new Path(dir + "testDataInMemoryWithoutWAL");
2246     FSHLog hLog = new FSHLog(fs, rootDir, "testDataInMemoryWithoutWAL", CONF);
2247     region = initHRegion(tableName, null, null, name.getMethodName(),
2248         CONF, false, Durability.SYNC_WAL, hLog, COLUMN_FAMILY_BYTES);
2249 
2250     Cell originalCell = CellUtil.createCell(row, COLUMN_FAMILY_BYTES, qual1,
2251       System.currentTimeMillis(), KeyValue.Type.Put.getCode(), value1);
2252     final long originalSize = KeyValueUtil.length(originalCell);
2253 
2254     Cell addCell = CellUtil.createCell(row, COLUMN_FAMILY_BYTES, qual1,
2255       System.currentTimeMillis(), KeyValue.Type.Put.getCode(), Bytes.toBytes("xxxxxxxxxx"));
2256     final long addSize = KeyValueUtil.length(addCell);
2257 
2258     LOG.info("originalSize:" + originalSize
2259       + ", addSize:" + addSize);
2260     // start test. We expect that the addPut's durability will be replaced
2261     // by originalPut's durability.
2262 
2263     // case 1:
2264     testDataInMemoryWithoutWAL(region,
2265             new Put(row).add(originalCell).setDurability(Durability.SKIP_WAL),
2266             new Put(row).add(addCell).setDurability(Durability.SKIP_WAL),
2267             originalSize + addSize);
2268 
2269     // case 2:
2270     testDataInMemoryWithoutWAL(region,
2271             new Put(row).add(originalCell).setDurability(Durability.SKIP_WAL),
2272             new Put(row).add(addCell).setDurability(Durability.SYNC_WAL),
2273             originalSize + addSize);
2274 
2275     // case 3:
2276     testDataInMemoryWithoutWAL(region,
2277             new Put(row).add(originalCell).setDurability(Durability.SYNC_WAL),
2278             new Put(row).add(addCell).setDurability(Durability.SKIP_WAL),
2279             0);
2280 
2281     // case 4:
2282     testDataInMemoryWithoutWAL(region,
2283             new Put(row).add(originalCell).setDurability(Durability.SYNC_WAL),
2284             new Put(row).add(addCell).setDurability(Durability.SYNC_WAL),
2285             0);
2286   }
2287 
2288   private static void testDataInMemoryWithoutWAL(HRegion region, Put originalPut,
2289           final Put addPut, long delta) throws IOException {
2290     final long initSize = region.getDataInMemoryWithoutWAL();
2291     // save normalCPHost and replaced by mockedCPHost
2292     RegionCoprocessorHost normalCPHost = region.getCoprocessorHost();
2293     RegionCoprocessorHost mockedCPHost = Mockito.mock(RegionCoprocessorHost.class);
2294     Answer<Boolean> answer = new Answer<Boolean>() {
2295       @Override
2296       public Boolean answer(InvocationOnMock invocation) throws Throwable {
2297         MiniBatchOperationInProgress<Mutation> mb = invocation.getArgumentAt(0,
2298                 MiniBatchOperationInProgress.class);
2299         mb.addOperationsFromCP(0, new Mutation[]{addPut});
2300         return false;
2301       }
2302     };
2303     when(mockedCPHost.preBatchMutate(Mockito.isA(MiniBatchOperationInProgress.class)))
2304       .then(answer);
2305     region.setCoprocessorHost(mockedCPHost);
2306     region.put(originalPut);
2307     region.setCoprocessorHost(normalCPHost);
2308     final long finalSize = region.getDataInMemoryWithoutWAL();
2309     assertEquals("finalSize:" + finalSize + ", initSize:"
2310       + initSize + ", delta:" + delta,finalSize, initSize + delta);
2311   }
2312 
2313   @Test
2314   public void testDeleteColumns_PostInsert() throws IOException, InterruptedException {
2315     Delete delete = new Delete(row);
2316     delete.deleteColumns(fam1, qual1);
2317     doTestDelete_AndPostInsert(delete);
2318   }
2319 
2320   @Test
2321   public void testDeleteFamily_PostInsert() throws IOException, InterruptedException {
2322     Delete delete = new Delete(row);
2323     delete.deleteFamily(fam1);
2324     doTestDelete_AndPostInsert(delete);
2325   }
2326 
2327   public void doTestDelete_AndPostInsert(Delete delete) throws IOException, InterruptedException {
2328     TableName tableName = TableName.valueOf(name.getMethodName());
2329     this.region = initHRegion(tableName, getName(), CONF, fam1);
2330     EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
2331     Put put = new Put(row);
2332     put.add(fam1, qual1, value1);
2333     region.put(put);
2334 
2335     // now delete the value:
2336     region.delete(delete);
2337 
2338     // ok put data:
2339     put = new Put(row);
2340     put.add(fam1, qual1, value2);
2341     region.put(put);
2342 
2343     // ok get:
2344     Get get = new Get(row);
2345     get.addColumn(fam1, qual1);
2346 
2347     Result r = region.get(get);
2348     assertEquals(1, r.size());
2349     assertArrayEquals(value2, r.getValue(fam1, qual1));
2350 
2351     // next:
2352     Scan scan = new Scan(row);
2353     scan.addColumn(fam1, qual1);
2354     InternalScanner s = region.getScanner(scan);
2355 
2356     List<Cell> results = new ArrayList<Cell>();
2357     assertEquals(false, s.next(results));
2358     assertEquals(1, results.size());
2359     Cell kv = results.get(0);
2360 
2361     assertArrayEquals(value2, CellUtil.cloneValue(kv));
2362     assertArrayEquals(fam1, CellUtil.cloneFamily(kv));
2363     assertArrayEquals(qual1, CellUtil.cloneQualifier(kv));
2364     assertArrayEquals(row, CellUtil.cloneRow(kv));
2365   }
2366 
2367   @Test
2368   public void testDelete_CheckTimestampUpdated() throws IOException {
2369     TableName tableName = TableName.valueOf(name.getMethodName());
2370     byte[] row1 = Bytes.toBytes("row1");
2371     byte[] col1 = Bytes.toBytes("col1");
2372     byte[] col2 = Bytes.toBytes("col2");
2373     byte[] col3 = Bytes.toBytes("col3");
2374 
2375     // Setting up region
2376     String method = this.getName();
2377     this.region = initHRegion(tableName, method, CONF, fam1);
2378     // Building checkerList
2379     List<Cell> kvs = new ArrayList<Cell>();
2380     kvs.add(new KeyValue(row1, fam1, col1, null));
2381     kvs.add(new KeyValue(row1, fam1, col2, null));
2382     kvs.add(new KeyValue(row1, fam1, col3, null));
2383 
2384     NavigableMap<byte[], List<Cell>> deleteMap = new TreeMap<byte[], List<Cell>>(
2385         Bytes.BYTES_COMPARATOR);
2386     deleteMap.put(fam1, kvs);
2387     region.delete(deleteMap, Durability.SYNC_WAL);
2388 
2389     // extract the key values out the memstore:
2390     // This is kinda hacky, but better than nothing...
2391     long now = System.currentTimeMillis();
2392     DefaultMemStore memstore = (DefaultMemStore) ((HStore) region.getStore(fam1)).memstore;
2393     Cell firstCell = memstore.activeSection.getCellSkipListSet().first();
2394     assertTrue(firstCell.getTimestamp() <= now);
2395     now = firstCell.getTimestamp();
2396     for (Cell cell : memstore.activeSection.getCellSkipListSet()) {
2397       assertTrue(cell.getTimestamp() <= now);
2398       now = cell.getTimestamp();
2399     }
2400   }
2401 
2402   // ////////////////////////////////////////////////////////////////////////////
2403   // Get tests
2404   // ////////////////////////////////////////////////////////////////////////////
2405   @Test
2406   public void testGet_FamilyChecker() throws IOException {
2407     byte[] row1 = Bytes.toBytes("row1");
2408     byte[] fam1 = Bytes.toBytes("fam1");
2409     byte[] fam2 = Bytes.toBytes("False");
2410     byte[] col1 = Bytes.toBytes("col1");
2411 
2412     // Setting up region
2413     String method = this.getName();
2414     this.region = initHRegion(tableName, method, CONF, fam1);
2415     Get get = new Get(row1);
2416     get.addColumn(fam2, col1);
2417 
2418     // Test
2419     try {
2420       region.get(get);
2421       fail("Expecting DoNotRetryIOException in get but did not get any");
2422     } catch (org.apache.hadoop.hbase.DoNotRetryIOException e) {
2423       LOG.info("Got expected DoNotRetryIOException successfully");
2424     }
2425   }
2426 
2427   @Test
2428   public void testGet_Basic() throws IOException {
2429     byte[] row1 = Bytes.toBytes("row1");
2430     byte[] fam1 = Bytes.toBytes("fam1");
2431     byte[] col1 = Bytes.toBytes("col1");
2432     byte[] col2 = Bytes.toBytes("col2");
2433     byte[] col3 = Bytes.toBytes("col3");
2434     byte[] col4 = Bytes.toBytes("col4");
2435     byte[] col5 = Bytes.toBytes("col5");
2436 
2437     // Setting up region
2438     String method = this.getName();
2439     this.region = initHRegion(tableName, method, CONF, fam1);
2440     // Add to memstore
2441     Put put = new Put(row1);
2442     put.add(fam1, col1, null);
2443     put.add(fam1, col2, null);
2444     put.add(fam1, col3, null);
2445     put.add(fam1, col4, null);
2446     put.add(fam1, col5, null);
2447     region.put(put);
2448 
2449     Get get = new Get(row1);
2450     get.addColumn(fam1, col2);
2451     get.addColumn(fam1, col4);
2452     // Expected result
2453     KeyValue kv1 = new KeyValue(row1, fam1, col2);
2454     KeyValue kv2 = new KeyValue(row1, fam1, col4);
2455     KeyValue[] expected = { kv1, kv2 };
2456 
2457     // Test
2458     Result res = region.get(get);
2459     assertEquals(expected.length, res.size());
2460     for (int i = 0; i < res.size(); i++) {
2461       assertTrue(CellUtil.matchingRow(expected[i], res.rawCells()[i]));
2462       assertTrue(CellUtil.matchingFamily(expected[i], res.rawCells()[i]));
2463       assertTrue(CellUtil.matchingQualifier(expected[i], res.rawCells()[i]));
2464     }
2465 
2466     // Test using a filter on a Get
2467     Get g = new Get(row1);
2468     final int count = 2;
2469     g.setFilter(new ColumnCountGetFilter(count));
2470     res = region.get(g);
2471     assertEquals(count, res.size());
2472   }
2473 
2474   @Test
2475   public void testGet_Empty() throws IOException {
2476     byte[] row = Bytes.toBytes("row");
2477     byte[] fam = Bytes.toBytes("fam");
2478 
2479     String method = this.getName();
2480     this.region = initHRegion(tableName, method, CONF, fam);
2481     Get get = new Get(row);
2482     get.addFamily(fam);
2483     Result r = region.get(get);
2484 
2485     assertTrue(r.isEmpty());
2486   }
2487 
2488   // ////////////////////////////////////////////////////////////////////////////
2489   // Merge test
2490   // ////////////////////////////////////////////////////////////////////////////
2491   @Test
2492   public void testMerge() throws IOException {
2493     byte[][] families = { fam1, fam2, fam3 };
2494     Configuration hc = initSplit();
2495     // Setting up region
2496     String method = this.getName();
2497     this.region = initHRegion(tableName, method, hc, families);
2498     LOG.info("" + HBaseTestCase.addContent(region, fam3));
2499     region.flush(true);
2500     region.compactStores();
2501     region.waitForFlushesAndCompactions();
2502     for(Store s:region.getStores()) {
2503       s.closeAndArchiveCompactedFiles();
2504     }
2505     byte[] splitRow = region.checkSplit();
2506     assertNotNull(splitRow);
2507     LOG.info("SplitRow: " + Bytes.toString(splitRow));
2508     HRegion[] subregions = splitRegion(region, splitRow);
2509     try {
2510       // Need to open the regions.
2511       for (int i = 0; i < subregions.length; i++) {
2512         HRegion.openHRegion(subregions[i], null);
2513         subregions[i].compactStores();
2514         subregions[i].waitForFlushesAndCompactions();
2515         for(Store s:subregions[i].getStores()) {
2516           s.closeAndArchiveCompactedFiles();
2517         }
2518       }
2519       Path oldRegionPath = region.getRegionFileSystem().getRegionDir();
2520       Path oldRegion1 = subregions[0].getRegionFileSystem().getRegionDir();
2521       Path oldRegion2 = subregions[1].getRegionFileSystem().getRegionDir();
2522       long startTime = System.currentTimeMillis();
2523       region = HRegion.mergeAdjacent(subregions[0], subregions[1]);
2524       LOG.info("Merge regions elapsed time: "
2525           + ((System.currentTimeMillis() - startTime) / 1000.0));
2526       FILESYSTEM.delete(oldRegion1, true);
2527       FILESYSTEM.delete(oldRegion2, true);
2528       FILESYSTEM.delete(oldRegionPath, true);
2529       LOG.info("splitAndMerge completed.");
2530     } finally {
2531       for (int i = 0; i < subregions.length; i++) {
2532         try {
2533           HRegion.closeHRegion(subregions[i]);
2534         } catch (IOException e) {
2535           // Ignore.
2536         }
2537       }
2538     }
2539   }
2540 
2541   /**
2542    * @param parent
2543    *          Region to split.
2544    * @param midkey
2545    *          Key to split around.
2546    * @return The Regions we created.
2547    * @throws IOException
2548    */
2549   HRegion[] splitRegion(final HRegion parent, final byte[] midkey) throws IOException {
2550     PairOfSameType<Region> result = null;
2551     SplitTransactionImpl st = new SplitTransactionImpl(parent, midkey);
2552     // If prepare does not return true, for some reason -- logged inside in
2553     // the prepare call -- we are not ready to split just now. Just return.
2554     if (!st.prepare()) {
2555       parent.clearSplit();
2556       return null;
2557     }
2558     try {
2559       result = st.execute(null, null);
2560     } catch (IOException ioe) {
2561       try {
2562         LOG.info("Running rollback of failed split of " +
2563           parent.getRegionInfo().getRegionNameAsString() + "; " + ioe.getMessage());
2564         st.rollback(null, null);
2565         LOG.info("Successful rollback of failed split of " +
2566           parent.getRegionInfo().getRegionNameAsString());
2567         return null;
2568       } catch (RuntimeException e) {
2569         // If failed rollback, kill this server to avoid having a hole in table.
2570         LOG.info("Failed rollback of failed split of " +
2571           parent.getRegionInfo().getRegionNameAsString() + " -- aborting server", e);
2572       }
2573     }
2574     finally {
2575       parent.clearSplit();
2576     }
2577 
2578     assertNotNull(result);
2579     return new HRegion[] { (HRegion)result.getFirst(), (HRegion)result.getSecond() };
2580   }
2581 
2582   // ////////////////////////////////////////////////////////////////////////////
2583   // Scanner tests
2584   // ////////////////////////////////////////////////////////////////////////////
2585   @Test
2586   public void testGetScanner_WithOkFamilies() throws IOException {
2587     byte[] fam1 = Bytes.toBytes("fam1");
2588     byte[] fam2 = Bytes.toBytes("fam2");
2589 
2590     byte[][] families = { fam1, fam2 };
2591 
2592     // Setting up region
2593     String method = this.getName();
2594     this.region = initHRegion(tableName, method, CONF, families);
2595     Scan scan = new Scan();
2596     scan.addFamily(fam1);
2597     scan.addFamily(fam2);
2598     try {
2599       region.getScanner(scan);
2600     } catch (Exception e) {
2601       assertTrue("Families could not be found in Region", false);
2602     }
2603   }
2604 
2605   @Test
2606   public void testGetScanner_WithNotOkFamilies() throws IOException {
2607     byte[] fam1 = Bytes.toBytes("fam1");
2608     byte[] fam2 = Bytes.toBytes("fam2");
2609 
2610     byte[][] families = { fam1 };
2611 
2612     // Setting up region
2613     String method = this.getName();
2614     this.region = initHRegion(tableName, method, CONF, families);
2615     Scan scan = new Scan();
2616     scan.addFamily(fam2);
2617     boolean ok = false;
2618     try {
2619       region.getScanner(scan);
2620     } catch (Exception e) {
2621       ok = true;
2622     }
2623     assertTrue("Families could not be found in Region", ok);
2624   }
2625 
2626   @Test
2627   public void testGetScanner_WithNoFamilies() throws IOException {
2628     byte[] row1 = Bytes.toBytes("row1");
2629     byte[] fam1 = Bytes.toBytes("fam1");
2630     byte[] fam2 = Bytes.toBytes("fam2");
2631     byte[] fam3 = Bytes.toBytes("fam3");
2632     byte[] fam4 = Bytes.toBytes("fam4");
2633 
2634     byte[][] families = { fam1, fam2, fam3, fam4 };
2635 
2636     // Setting up region
2637     String method = this.getName();
2638     this.region = initHRegion(tableName, method, CONF, families);
2639     // Putting data in Region
2640     Put put = new Put(row1);
2641     put.add(fam1, null, null);
2642     put.add(fam2, null, null);
2643     put.add(fam3, null, null);
2644     put.add(fam4, null, null);
2645     region.put(put);
2646 
2647     Scan scan = null;
2648     HRegion.RegionScannerImpl is = null;
2649 
2650     // Testing to see how many scanners that is produced by getScanner,
2651     // starting
2652     // with known number, 2 - current = 1
2653     scan = new Scan();
2654     scan.addFamily(fam2);
2655     scan.addFamily(fam4);
2656     is = (RegionScannerImpl) region.getScanner(scan);
2657     assertEquals(1, ((RegionScannerImpl) is).storeHeap.getHeap().size());
2658 
2659     scan = new Scan();
2660     is = (RegionScannerImpl) region.getScanner(scan);
2661     assertEquals(families.length - 1, ((RegionScannerImpl) is).storeHeap.getHeap().size());
2662   }
2663 
2664   /**
2665    * This method tests https://issues.apache.org/jira/browse/HBASE-2516.
2666    *
2667    * @throws IOException
2668    */
2669   @Test
2670   public void testGetScanner_WithRegionClosed() throws IOException {
2671     byte[] fam1 = Bytes.toBytes("fam1");
2672     byte[] fam2 = Bytes.toBytes("fam2");
2673 
2674     byte[][] families = { fam1, fam2 };
2675 
2676     // Setting up region
2677     String method = this.getName();
2678     try {
2679       this.region = initHRegion(tableName, method, CONF, families);
2680     } catch (IOException e) {
2681       e.printStackTrace();
2682       fail("Got IOException during initHRegion, " + e.getMessage());
2683     }
2684     region.closed.set(true);
2685     try {
2686       region.getScanner(new Scan());
2687       fail("Expected to get an exception during getScanner on a region that is closed");
2688     } catch (NotServingRegionException e) {
2689       // this is the correct exception that is expected
2690     } catch (IOException e) {
2691       fail("Got wrong type of exception - should be a NotServingRegionException, but was an IOException: "
2692           + e.getMessage());
2693     }
2694   }
2695 
2696   @Test
2697   public void testRegionScanner_Next() throws IOException {
2698     byte[] row1 = Bytes.toBytes("row1");
2699     byte[] row2 = Bytes.toBytes("row2");
2700     byte[] fam1 = Bytes.toBytes("fam1");
2701     byte[] fam2 = Bytes.toBytes("fam2");
2702     byte[] fam3 = Bytes.toBytes("fam3");
2703     byte[] fam4 = Bytes.toBytes("fam4");
2704 
2705     byte[][] families = { fam1, fam2, fam3, fam4 };
2706     long ts = System.currentTimeMillis();
2707 
2708     // Setting up region
2709     String method = this.getName();
2710     this.region = initHRegion(tableName, method, CONF, families);
2711     // Putting data in Region
2712     Put put = null;
2713     put = new Put(row1);
2714     put.add(fam1, (byte[]) null, ts, null);
2715     put.add(fam2, (byte[]) null, ts, null);
2716     put.add(fam3, (byte[]) null, ts, null);
2717     put.add(fam4, (byte[]) null, ts, null);
2718     region.put(put);
2719 
2720     put = new Put(row2);
2721     put.add(fam1, (byte[]) null, ts, null);
2722     put.add(fam2, (byte[]) null, ts, null);
2723     put.add(fam3, (byte[]) null, ts, null);
2724     put.add(fam4, (byte[]) null, ts, null);
2725     region.put(put);
2726 
2727     Scan scan = new Scan();
2728     scan.addFamily(fam2);
2729     scan.addFamily(fam4);
2730     InternalScanner is = region.getScanner(scan);
2731 
2732     List<Cell> res = null;
2733 
2734     // Result 1
2735     List<Cell> expected1 = new ArrayList<Cell>();
2736     expected1.add(new KeyValue(row1, fam2, null, ts, KeyValue.Type.Put, null));
2737     expected1.add(new KeyValue(row1, fam4, null, ts, KeyValue.Type.Put, null));
2738 
2739     res = new ArrayList<Cell>();
2740     is.next(res);
2741     for (int i = 0; i < res.size(); i++) {
2742       assertTrue(CellComparator.equalsIgnoreMvccVersion(expected1.get(i), res.get(i)));
2743     }
2744 
2745     // Result 2
2746     List<Cell> expected2 = new ArrayList<Cell>();
2747     expected2.add(new KeyValue(row2, fam2, null, ts, KeyValue.Type.Put, null));
2748     expected2.add(new KeyValue(row2, fam4, null, ts, KeyValue.Type.Put, null));
2749 
2750     res = new ArrayList<Cell>();
2751     is.next(res);
2752     for (int i = 0; i < res.size(); i++) {
2753       assertTrue(CellComparator.equalsIgnoreMvccVersion(expected2.get(i), res.get(i)));
2754     }
2755   }
2756 
2757   @Test
2758   public void testScanner_ExplicitColumns_FromMemStore_EnforceVersions() throws IOException {
2759     byte[] row1 = Bytes.toBytes("row1");
2760     byte[] qf1 = Bytes.toBytes("qualifier1");
2761     byte[] qf2 = Bytes.toBytes("qualifier2");
2762     byte[] fam1 = Bytes.toBytes("fam1");
2763     byte[][] families = { fam1 };
2764 
2765     long ts1 = System.currentTimeMillis();
2766     long ts2 = ts1 + 1;
2767     long ts3 = ts1 + 2;
2768 
2769     // Setting up region
2770     String method = this.getName();
2771     this.region = initHRegion(tableName, method, CONF, families);
2772     // Putting data in Region
2773     Put put = null;
2774     KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2775     KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2776     KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2777 
2778     KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2779     KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2780     KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2781 
2782     put = new Put(row1);
2783     put.add(kv13);
2784     put.add(kv12);
2785     put.add(kv11);
2786     put.add(kv23);
2787     put.add(kv22);
2788     put.add(kv21);
2789     region.put(put);
2790 
2791     // Expected
2792     List<Cell> expected = new ArrayList<Cell>();
2793     expected.add(kv13);
2794     expected.add(kv12);
2795 
2796     Scan scan = new Scan(row1);
2797     scan.addColumn(fam1, qf1);
2798     scan.setMaxVersions(MAX_VERSIONS);
2799     List<Cell> actual = new ArrayList<Cell>();
2800     InternalScanner scanner = region.getScanner(scan);
2801 
2802     boolean hasNext = scanner.next(actual);
2803     assertEquals(false, hasNext);
2804 
2805     // Verify result
2806     for (int i = 0; i < expected.size(); i++) {
2807       assertEquals(expected.get(i), actual.get(i));
2808     }
2809   }
2810 
2811   @Test
2812   public void testScanner_ExplicitColumns_FromFilesOnly_EnforceVersions() throws IOException {
2813     byte[] row1 = Bytes.toBytes("row1");
2814     byte[] qf1 = Bytes.toBytes("qualifier1");
2815     byte[] qf2 = Bytes.toBytes("qualifier2");
2816     byte[] fam1 = Bytes.toBytes("fam1");
2817     byte[][] families = { fam1 };
2818 
2819     long ts1 = 1; // System.currentTimeMillis();
2820     long ts2 = ts1 + 1;
2821     long ts3 = ts1 + 2;
2822 
2823     // Setting up region
2824     String method = this.getName();
2825     this.region = initHRegion(tableName, method, CONF, families);
2826     // Putting data in Region
2827     Put put = null;
2828     KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2829     KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2830     KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2831 
2832     KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2833     KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2834     KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2835 
2836     put = new Put(row1);
2837     put.add(kv13);
2838     put.add(kv12);
2839     put.add(kv11);
2840     put.add(kv23);
2841     put.add(kv22);
2842     put.add(kv21);
2843     region.put(put);
2844     region.flush(true);
2845 
2846     // Expected
2847     List<Cell> expected = new ArrayList<Cell>();
2848     expected.add(kv13);
2849     expected.add(kv12);
2850     expected.add(kv23);
2851     expected.add(kv22);
2852 
2853     Scan scan = new Scan(row1);
2854     scan.addColumn(fam1, qf1);
2855     scan.addColumn(fam1, qf2);
2856     scan.setMaxVersions(MAX_VERSIONS);
2857     List<Cell> actual = new ArrayList<Cell>();
2858     InternalScanner scanner = region.getScanner(scan);
2859 
2860     boolean hasNext = scanner.next(actual);
2861     assertEquals(false, hasNext);
2862 
2863     // Verify result
2864     for (int i = 0; i < expected.size(); i++) {
2865       assertTrue(CellComparator.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
2866     }
2867   }
2868 
2869   @Test
2870   public void testScanner_ExplicitColumns_FromMemStoreAndFiles_EnforceVersions() throws IOException {
2871     byte[] row1 = Bytes.toBytes("row1");
2872     byte[] fam1 = Bytes.toBytes("fam1");
2873     byte[][] families = { fam1 };
2874     byte[] qf1 = Bytes.toBytes("qualifier1");
2875     byte[] qf2 = Bytes.toBytes("qualifier2");
2876 
2877     long ts1 = 1;
2878     long ts2 = ts1 + 1;
2879     long ts3 = ts1 + 2;
2880     long ts4 = ts1 + 3;
2881 
2882     // Setting up region
2883     String method = this.getName();
2884     this.region = initHRegion(tableName, method, CONF, families);
2885     // Putting data in Region
2886     KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
2887     KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2888     KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2889     KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2890 
2891     KeyValue kv24 = new KeyValue(row1, fam1, qf2, ts4, KeyValue.Type.Put, null);
2892     KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2893     KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2894     KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2895 
2896     Put put = null;
2897     put = new Put(row1);
2898     put.add(kv14);
2899     put.add(kv24);
2900     region.put(put);
2901     region.flush(true);
2902 
2903     put = new Put(row1);
2904     put.add(kv23);
2905     put.add(kv13);
2906     region.put(put);
2907     region.flush(true);
2908 
2909     put = new Put(row1);
2910     put.add(kv22);
2911     put.add(kv12);
2912     region.put(put);
2913     region.flush(true);
2914 
2915     put = new Put(row1);
2916     put.add(kv21);
2917     put.add(kv11);
2918     region.put(put);
2919 
2920     // Expected
2921     List<Cell> expected = new ArrayList<Cell>();
2922     expected.add(kv14);
2923     expected.add(kv13);
2924     expected.add(kv12);
2925     expected.add(kv24);
2926     expected.add(kv23);
2927     expected.add(kv22);
2928 
2929     Scan scan = new Scan(row1);
2930     scan.addColumn(fam1, qf1);
2931     scan.addColumn(fam1, qf2);
2932     int versions = 3;
2933     scan.setMaxVersions(versions);
2934     List<Cell> actual = new ArrayList<Cell>();
2935     InternalScanner scanner = region.getScanner(scan);
2936 
2937     boolean hasNext = scanner.next(actual);
2938     assertEquals(false, hasNext);
2939 
2940     // Verify result
2941     for (int i = 0; i < expected.size(); i++) {
2942       assertTrue(CellComparator.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
2943     }
2944   }
2945 
2946   @Test
2947   public void testScanner_Wildcard_FromMemStore_EnforceVersions() throws IOException {
2948     byte[] row1 = Bytes.toBytes("row1");
2949     byte[] qf1 = Bytes.toBytes("qualifier1");
2950     byte[] qf2 = Bytes.toBytes("qualifier2");
2951     byte[] fam1 = Bytes.toBytes("fam1");
2952     byte[][] families = { fam1 };
2953 
2954     long ts1 = System.currentTimeMillis();
2955     long ts2 = ts1 + 1;
2956     long ts3 = ts1 + 2;
2957 
2958     // Setting up region
2959     String method = this.getName();
2960     this.region = initHRegion(tableName, method, CONF, families);
2961     // Putting data in Region
2962     Put put = null;
2963     KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2964     KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2965     KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2966 
2967     KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2968     KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2969     KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2970 
2971     put = new Put(row1);
2972     put.add(kv13);
2973     put.add(kv12);
2974     put.add(kv11);
2975     put.add(kv23);
2976     put.add(kv22);
2977     put.add(kv21);
2978     region.put(put);
2979 
2980     // Expected
2981     List<Cell> expected = new ArrayList<Cell>();
2982     expected.add(kv13);
2983     expected.add(kv12);
2984     expected.add(kv23);
2985     expected.add(kv22);
2986 
2987     Scan scan = new Scan(row1);
2988     scan.addFamily(fam1);
2989     scan.setMaxVersions(MAX_VERSIONS);
2990     List<Cell> actual = new ArrayList<Cell>();
2991     InternalScanner scanner = region.getScanner(scan);
2992 
2993     boolean hasNext = scanner.next(actual);
2994     assertEquals(false, hasNext);
2995 
2996     // Verify result
2997     for (int i = 0; i < expected.size(); i++) {
2998       assertEquals(expected.get(i), actual.get(i));
2999     }
3000   }
3001 
3002   @Test
3003   public void testScanner_Wildcard_FromFilesOnly_EnforceVersions() throws IOException {
3004     byte[] row1 = Bytes.toBytes("row1");
3005     byte[] qf1 = Bytes.toBytes("qualifier1");
3006     byte[] qf2 = Bytes.toBytes("qualifier2");
3007     byte[] fam1 = Bytes.toBytes("fam1");
3008 
3009     long ts1 = 1; // System.currentTimeMillis();
3010     long ts2 = ts1 + 1;
3011     long ts3 = ts1 + 2;
3012 
3013     // Setting up region
3014     String method = this.getName();
3015     this.region = initHRegion(tableName, method, CONF, fam1);
3016     // Putting data in Region
3017     Put put = null;
3018     KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
3019     KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
3020     KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
3021 
3022     KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
3023     KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
3024     KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
3025 
3026     put = new Put(row1);
3027     put.add(kv13);
3028     put.add(kv12);
3029     put.add(kv11);
3030     put.add(kv23);
3031     put.add(kv22);
3032     put.add(kv21);
3033     region.put(put);
3034     region.flush(true);
3035 
3036     // Expected
3037     List<Cell> expected = new ArrayList<Cell>();
3038     expected.add(kv13);
3039     expected.add(kv12);
3040     expected.add(kv23);
3041     expected.add(kv22);
3042 
3043     Scan scan = new Scan(row1);
3044     scan.addFamily(fam1);
3045     scan.setMaxVersions(MAX_VERSIONS);
3046     List<Cell> actual = new ArrayList<Cell>();
3047     InternalScanner scanner = region.getScanner(scan);
3048 
3049     boolean hasNext = scanner.next(actual);
3050     assertEquals(false, hasNext);
3051 
3052     // Verify result
3053     for (int i = 0; i < expected.size(); i++) {
3054       assertTrue(CellComparator.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
3055     }
3056   }
3057 
3058   @Test
3059   public void testScanner_StopRow1542() throws IOException {
3060     byte[] family = Bytes.toBytes("testFamily");
3061     this.region = initHRegion(tableName, getName(), CONF, family);
3062     byte[] row1 = Bytes.toBytes("row111");
3063     byte[] row2 = Bytes.toBytes("row222");
3064     byte[] row3 = Bytes.toBytes("row333");
3065     byte[] row4 = Bytes.toBytes("row444");
3066     byte[] row5 = Bytes.toBytes("row555");
3067 
3068     byte[] col1 = Bytes.toBytes("Pub111");
3069     byte[] col2 = Bytes.toBytes("Pub222");
3070 
3071     Put put = new Put(row1);
3072     put.add(family, col1, Bytes.toBytes(10L));
3073     region.put(put);
3074 
3075     put = new Put(row2);
3076     put.add(family, col1, Bytes.toBytes(15L));
3077     region.put(put);
3078 
3079     put = new Put(row3);
3080     put.add(family, col2, Bytes.toBytes(20L));
3081     region.put(put);
3082 
3083     put = new Put(row4);
3084     put.add(family, col2, Bytes.toBytes(30L));
3085     region.put(put);
3086 
3087     put = new Put(row5);
3088     put.add(family, col1, Bytes.toBytes(40L));
3089     region.put(put);
3090 
3091     Scan scan = new Scan(row3, row4);
3092     scan.setMaxVersions();
3093     scan.addColumn(family, col1);
3094     InternalScanner s = region.getScanner(scan);
3095 
3096     List<Cell> results = new ArrayList<Cell>();
3097     assertEquals(false, s.next(results));
3098     assertEquals(0, results.size());
3099   }
3100 
3101   @Test
3102   public void testScanner_Wildcard_FromMemStoreAndFiles_EnforceVersions() throws IOException {
3103     byte[] row1 = Bytes.toBytes("row1");
3104     byte[] fam1 = Bytes.toBytes("fam1");
3105     byte[] qf1 = Bytes.toBytes("qualifier1");
3106     byte[] qf2 = Bytes.toBytes("quateslifier2");
3107 
3108     long ts1 = 1;
3109     long ts2 = ts1 + 1;
3110     long ts3 = ts1 + 2;
3111     long ts4 = ts1 + 3;
3112 
3113     // Setting up region
3114     String method = this.getName();
3115     this.region = initHRegion(tableName, method, CONF, fam1);
3116     // Putting data in Region
3117     KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
3118     KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
3119     KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
3120     KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
3121 
3122     KeyValue kv24 = new KeyValue(row1, fam1, qf2, ts4, KeyValue.Type.Put, null);
3123     KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
3124     KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
3125     KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
3126 
3127     Put put = null;
3128     put = new Put(row1);
3129     put.add(kv14);
3130     put.add(kv24);
3131     region.put(put);
3132     region.flush(true);
3133 
3134     put = new Put(row1);
3135     put.add(kv23);
3136     put.add(kv13);
3137     region.put(put);
3138     region.flush(true);
3139 
3140     put = new Put(row1);
3141     put.add(kv22);
3142     put.add(kv12);
3143     region.put(put);
3144     region.flush(true);
3145 
3146     put = new Put(row1);
3147     put.add(kv21);
3148     put.add(kv11);
3149     region.put(put);
3150 
3151     // Expected
3152     List<KeyValue> expected = new ArrayList<KeyValue>();
3153     expected.add(kv14);
3154     expected.add(kv13);
3155     expected.add(kv12);
3156     expected.add(kv24);
3157     expected.add(kv23);
3158     expected.add(kv22);
3159 
3160     Scan scan = new Scan(row1);
3161     int versions = 3;
3162     scan.setMaxVersions(versions);
3163     List<Cell> actual = new ArrayList<Cell>();
3164     InternalScanner scanner = region.getScanner(scan);
3165 
3166     boolean hasNext = scanner.next(actual);
3167     assertEquals(false, hasNext);
3168 
3169     // Verify result
3170     for (int i = 0; i < expected.size(); i++) {
3171       assertTrue(CellComparator.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
3172     }
3173   }
3174 
3175   /**
3176    * Added for HBASE-5416
3177    *
3178    * Here we test scan optimization when only subset of CFs are used in filter
3179    * conditions.
3180    */
3181   @Test
3182   public void testScanner_JoinedScanners() throws IOException {
3183     byte[] cf_essential = Bytes.toBytes("essential");
3184     byte[] cf_joined = Bytes.toBytes("joined");
3185     byte[] cf_alpha = Bytes.toBytes("alpha");
3186     this.region = initHRegion(tableName, getName(), CONF, cf_essential, cf_joined, cf_alpha);
3187     byte[] row1 = Bytes.toBytes("row1");
3188     byte[] row2 = Bytes.toBytes("row2");
3189     byte[] row3 = Bytes.toBytes("row3");
3190 
3191     byte[] col_normal = Bytes.toBytes("d");
3192     byte[] col_alpha = Bytes.toBytes("a");
3193 
3194     byte[] filtered_val = Bytes.toBytes(3);
3195 
3196     Put put = new Put(row1);
3197     put.add(cf_essential, col_normal, Bytes.toBytes(1));
3198     put.add(cf_joined, col_alpha, Bytes.toBytes(1));
3199     region.put(put);
3200 
3201     put = new Put(row2);
3202     put.add(cf_essential, col_alpha, Bytes.toBytes(2));
3203     put.add(cf_joined, col_normal, Bytes.toBytes(2));
3204     put.add(cf_alpha, col_alpha, Bytes.toBytes(2));
3205     region.put(put);
3206 
3207     put = new Put(row3);
3208     put.add(cf_essential, col_normal, filtered_val);
3209     put.add(cf_joined, col_normal, filtered_val);
3210     region.put(put);
3211 
3212     // Check two things:
3213     // 1. result list contains expected values
3214     // 2. result list is sorted properly
3215 
3216     Scan scan = new Scan();
3217     Filter filter = new SingleColumnValueExcludeFilter(cf_essential, col_normal,
3218         CompareOp.NOT_EQUAL, filtered_val);
3219     scan.setFilter(filter);
3220     scan.setLoadColumnFamiliesOnDemand(true);
3221     InternalScanner s = region.getScanner(scan);
3222 
3223     List<Cell> results = new ArrayList<Cell>();
3224     assertTrue(s.next(results));
3225     assertEquals(results.size(), 1);
3226     results.clear();
3227 
3228     assertTrue(s.next(results));
3229     assertEquals(results.size(), 3);
3230     assertTrue("orderCheck", CellUtil.matchingFamily(results.get(0), cf_alpha));
3231     assertTrue("orderCheck", CellUtil.matchingFamily(results.get(1), cf_essential));
3232     assertTrue("orderCheck", CellUtil.matchingFamily(results.get(2), cf_joined));
3233     results.clear();
3234 
3235     assertFalse(s.next(results));
3236     assertEquals(results.size(), 0);
3237   }
3238 
3239   /**
3240    * HBASE-5416
3241    *
3242    * Test case when scan limits amount of KVs returned on each next() call.
3243    */
3244   @Test
3245   public void testScanner_JoinedScannersWithLimits() throws IOException {
3246     final byte[] cf_first = Bytes.toBytes("first");
3247     final byte[] cf_second = Bytes.toBytes("second");
3248 
3249     this.region = initHRegion(tableName, getName(), CONF, cf_first, cf_second);
3250     final byte[] col_a = Bytes.toBytes("a");
3251     final byte[] col_b = Bytes.toBytes("b");
3252 
3253     Put put;
3254 
3255     for (int i = 0; i < 10; i++) {
3256       put = new Put(Bytes.toBytes("r" + Integer.toString(i)));
3257       put.add(cf_first, col_a, Bytes.toBytes(i));
3258       if (i < 5) {
3259         put.add(cf_first, col_b, Bytes.toBytes(i));
3260         put.add(cf_second, col_a, Bytes.toBytes(i));
3261         put.add(cf_second, col_b, Bytes.toBytes(i));
3262       }
3263       region.put(put);
3264     }
3265 
3266     Scan scan = new Scan();
3267     scan.setLoadColumnFamiliesOnDemand(true);
3268     Filter bogusFilter = new FilterBase() {
3269       @Override
3270       public ReturnCode filterKeyValue(Cell ignored) throws IOException {
3271         return ReturnCode.INCLUDE;
3272       }
3273       @Override
3274       public boolean isFamilyEssential(byte[] name) {
3275         return Bytes.equals(name, cf_first);
3276       }
3277     };
3278 
3279     scan.setFilter(bogusFilter);
3280     InternalScanner s = region.getScanner(scan);
3281 
3282     // Our data looks like this:
3283     // r0: first:a, first:b, second:a, second:b
3284     // r1: first:a, first:b, second:a, second:b
3285     // r2: first:a, first:b, second:a, second:b
3286     // r3: first:a, first:b, second:a, second:b
3287     // r4: first:a, first:b, second:a, second:b
3288     // r5: first:a
3289     // r6: first:a
3290     // r7: first:a
3291     // r8: first:a
3292     // r9: first:a
3293 
3294     // But due to next's limit set to 3, we should get this:
3295     // r0: first:a, first:b, second:a
3296     // r0: second:b
3297     // r1: first:a, first:b, second:a
3298     // r1: second:b
3299     // r2: first:a, first:b, second:a
3300     // r2: second:b
3301     // r3: first:a, first:b, second:a
3302     // r3: second:b
3303     // r4: first:a, first:b, second:a
3304     // r4: second:b
3305     // r5: first:a
3306     // r6: first:a
3307     // r7: first:a
3308     // r8: first:a
3309     // r9: first:a
3310 
3311     List<Cell> results = new ArrayList<Cell>();
3312     int index = 0;
3313     ScannerContext scannerContext = ScannerContext.newBuilder().setBatchLimit(3).build();
3314     while (true) {
3315       boolean more = s.next(results, scannerContext);
3316       if ((index >> 1) < 5) {
3317         if (index % 2 == 0)
3318           assertEquals(results.size(), 3);
3319         else
3320           assertEquals(results.size(), 1);
3321       } else
3322         assertEquals(results.size(), 1);
3323       results.clear();
3324       index++;
3325       if (!more)
3326         break;
3327     }
3328   }
3329 
3330   /**
3331    * Write an HFile block full with Cells whose qualifier that are identical between
3332    * 0 and Short.MAX_VALUE. See HBASE-13329.
3333    * @throws Exception
3334    */
3335   @Test
3336   public void testLongQualifier() throws Exception {
3337     String method = name.getMethodName();
3338     TableName tableName = TableName.valueOf(method);
3339     byte[] family = Bytes.toBytes("family");
3340     this.region = initHRegion(tableName, method, CONF, family);
3341     byte[] q = new byte[Short.MAX_VALUE+2];
3342     Arrays.fill(q, 0, q.length-1, (byte)42);
3343     for (byte i=0; i<10; i++) {
3344       Put p = new Put(Bytes.toBytes("row"));
3345       // qualifiers that differ past Short.MAX_VALUE
3346       q[q.length-1]=i;
3347       p.addColumn(family, q, q);
3348       region.put(p);
3349     }
3350     region.flush(false);
3351   }
3352 
3353   // ////////////////////////////////////////////////////////////////////////////
3354   // Split test
3355   // ////////////////////////////////////////////////////////////////////////////
3356   /**
3357    * Splits twice and verifies getting from each of the split regions.
3358    *
3359    * @throws Exception
3360    */
3361   @Test
3362   public void testBasicSplit() throws Exception {
3363     byte[][] families = { fam1, fam2, fam3 };
3364 
3365     Configuration hc = initSplit();
3366     // Setting up region
3367     String method = this.getName();
3368     this.region = initHRegion(tableName, method, hc, families);
3369 
3370     LOG.info("" + HBaseTestCase.addContent(region, fam3));
3371     region.flush(true);
3372     region.compactStores();
3373     byte[] splitRow = region.checkSplit();
3374     assertNotNull(splitRow);
3375     LOG.info("SplitRow: " + Bytes.toString(splitRow));
3376     HRegion[] regions = splitRegion(region, splitRow);
3377     try {
3378       // Need to open the regions.
3379       // TODO: Add an 'open' to HRegion... don't do open by constructing
3380       // instance.
3381       for (int i = 0; i < regions.length; i++) {
3382         regions[i] = HRegion.openHRegion(regions[i], null);
3383       }
3384       // Assert can get rows out of new regions. Should be able to get first
3385       // row from first region and the midkey from second region.
3386       assertGet(regions[0], fam3, Bytes.toBytes(START_KEY));
3387       assertGet(regions[1], fam3, splitRow);
3388       // Test I can get scanner and that it starts at right place.
3389       assertScan(regions[0], fam3, Bytes.toBytes(START_KEY));
3390       assertScan(regions[1], fam3, splitRow);
3391       // Now prove can't split regions that have references.
3392       for (int i = 0; i < regions.length; i++) {
3393         // Add so much data to this region, we create a store file that is >
3394         // than one of our unsplitable references. it will.
3395         for (int j = 0; j < 2; j++) {
3396           HBaseTestCase.addContent(regions[i], fam3);
3397         }
3398         HBaseTestCase.addContent(regions[i], fam2);
3399         HBaseTestCase.addContent(regions[i], fam1);
3400         regions[i].flush(true);
3401       }
3402 
3403       byte[][] midkeys = new byte[regions.length][];
3404       // To make regions splitable force compaction.
3405       for (int i = 0; i < regions.length; i++) {
3406         regions[i].compactStores();
3407         midkeys[i] = regions[i].checkSplit();
3408       }
3409 
3410       TreeMap<String, HRegion> sortedMap = new TreeMap<String, HRegion>();
3411       // Split these two daughter regions so then I'll have 4 regions. Will
3412       // split because added data above.
3413       for (int i = 0; i < regions.length; i++) {
3414         HRegion[] rs = null;
3415         if (midkeys[i] != null) {
3416           rs = splitRegion(regions[i], midkeys[i]);
3417           for (int j = 0; j < rs.length; j++) {
3418             sortedMap.put(Bytes.toString(rs[j].getRegionInfo().getRegionName()),
3419               HRegion.openHRegion(rs[j], null));
3420           }
3421         }
3422       }
3423       LOG.info("Made 4 regions");
3424       // The splits should have been even. Test I can get some arbitrary row
3425       // out of each.
3426       int interval = (LAST_CHAR - FIRST_CHAR) / 3;
3427       byte[] b = Bytes.toBytes(START_KEY);
3428       for (HRegion r : sortedMap.values()) {
3429         assertGet(r, fam3, b);
3430         b[0] += interval;
3431       }
3432     } finally {
3433       for (int i = 0; i < regions.length; i++) {
3434         try {
3435           regions[i].close();
3436         } catch (IOException e) {
3437           // Ignore.
3438         }
3439       }
3440     }
3441   }
3442 
3443   @Test
3444   public void testSplitRegion() throws IOException {
3445     byte[] qualifier = Bytes.toBytes("qualifier");
3446     Configuration hc = initSplit();
3447     int numRows = 10;
3448     byte[][] families = { fam1, fam3 };
3449 
3450     // Setting up region
3451     String method = this.getName();
3452     this.region = initHRegion(tableName, method, hc, families);
3453 
3454     // Put data in region
3455     int startRow = 100;
3456     putData(startRow, numRows, qualifier, families);
3457     int splitRow = startRow + numRows;
3458     putData(splitRow, numRows, qualifier, families);
3459     region.flush(true);
3460 
3461     HRegion[] regions = null;
3462     regions = splitRegion(region, Bytes.toBytes("" + splitRow));
3463     // Opening the regions returned.
3464     for (int i = 0; i < regions.length; i++) {
3465       regions[i] = HRegion.openHRegion(regions[i], null);
3466     }
3467     // Verifying that the region has been split
3468     assertEquals(2, regions.length);
3469 
3470     // Verifying that all data is still there and that data is in the right
3471     // place
3472     verifyData(regions[0], startRow, numRows, qualifier, families);
3473     verifyData(regions[1], splitRow, numRows, qualifier, families);
3474   }
3475 
3476   @Test
3477   public void testClearForceSplit() throws IOException {
3478     byte[] qualifier = Bytes.toBytes("qualifier");
3479     Configuration hc = initSplit();
3480     int numRows = 10;
3481     byte[][] families = { fam1, fam3 };
3482 
3483     // Setting up region
3484     String method = this.getName();
3485     this.region = initHRegion(tableName, method, hc, families);
3486 
3487     // Put data in region
3488     int startRow = 100;
3489     putData(startRow, numRows, qualifier, families);
3490     int splitRow = startRow + numRows;
3491     byte[] splitRowBytes = Bytes.toBytes("" + splitRow);
3492     putData(splitRow, numRows, qualifier, families);
3493     region.flush(true);
3494 
3495     HRegion[] regions = null;
3496     // Set force split
3497     region.forceSplit(splitRowBytes);
3498     assertTrue(region.shouldForceSplit());
3499     // Split point should be the force split row
3500     assertTrue(Bytes.equals(splitRowBytes, region.checkSplit()));
3501 
3502     // Add a store that has references.
3503     HStore storeMock = Mockito.mock(HStore.class);
3504     when(storeMock.hasReferences()).thenReturn(true);
3505     when(storeMock.getFamily()).thenReturn(new HColumnDescriptor("cf"));
3506     when(storeMock.close()).thenReturn(ImmutableList.<StoreFile>of());
3507     when(storeMock.getColumnFamilyName()).thenReturn("cf");
3508     region.stores.put(Bytes.toBytes(storeMock.getColumnFamilyName()), storeMock);
3509     assertTrue(region.hasReferences());
3510 
3511     // Will not split since the store has references.
3512     regions = splitRegion(region, splitRowBytes);
3513     assertNull(regions);
3514 
3515     // Region force split should be cleared after the split try.
3516     assertFalse(region.shouldForceSplit());
3517 
3518     // Remove the store that has references.
3519     region.stores.remove(Bytes.toBytes(storeMock.getColumnFamilyName()));
3520     assertFalse(region.hasReferences());
3521 
3522     // Now we can split.
3523     regions = splitRegion(region, splitRowBytes);
3524 
3525     // Opening the regions returned.
3526     for (int i = 0; i < regions.length; i++) {
3527       regions[i] = HRegion.openHRegion(regions[i], null);
3528     }
3529     // Verifying that the region has been split
3530     assertEquals(2, regions.length);
3531 
3532     // Verifying that all data is still there and that data is in the right
3533     // place
3534     verifyData(regions[0], startRow, numRows, qualifier, families);
3535     verifyData(regions[1], splitRow, numRows, qualifier, families);
3536   }
3537 
3538   /**
3539    * Flushes the cache in a thread while scanning. The tests verify that the
3540    * scan is coherent - e.g. the returned results are always of the same or
3541    * later update as the previous results.
3542    *
3543    * @throws IOException
3544    *           scan / compact
3545    * @throws InterruptedException
3546    *           thread join
3547    */
3548   @Test
3549   public void testFlushCacheWhileScanning() throws IOException, InterruptedException {
3550     byte[] family = Bytes.toBytes("family");
3551     int numRows = 1000;
3552     int flushAndScanInterval = 10;
3553     int compactInterval = 10 * flushAndScanInterval;
3554 
3555     String method = "testFlushCacheWhileScanning";
3556     this.region = initHRegion(tableName, method, CONF, family);
3557     FlushThread flushThread = new FlushThread();
3558     try {
3559       flushThread.start();
3560 
3561       Scan scan = new Scan();
3562       scan.addFamily(family);
3563       scan.setFilter(new SingleColumnValueFilter(family, qual1, CompareOp.EQUAL,
3564           new BinaryComparator(Bytes.toBytes(5L))));
3565 
3566       int expectedCount = 0;
3567       List<Cell> res = new ArrayList<Cell>();
3568 
3569       boolean toggle = true;
3570       for (long i = 0; i < numRows; i++) {
3571         Put put = new Put(Bytes.toBytes(i));
3572         put.setDurability(Durability.SKIP_WAL);
3573         put.add(family, qual1, Bytes.toBytes(i % 10));
3574         region.put(put);
3575 
3576         if (i != 0 && i % compactInterval == 0) {
3577           LOG.debug("iteration = " + i+ " ts="+System.currentTimeMillis());
3578           region.compact(true);
3579         }
3580 
3581         if (i % 10 == 5L) {
3582           expectedCount++;
3583         }
3584 
3585         if (i != 0 && i % flushAndScanInterval == 0) {
3586           res.clear();
3587           InternalScanner scanner = region.getScanner(scan);
3588           if (toggle) {
3589             flushThread.flush();
3590           }
3591           while (scanner.next(res))
3592             ;
3593           if (!toggle) {
3594             flushThread.flush();
3595           }
3596           assertEquals("toggle="+toggle+"i=" + i + " ts="+System.currentTimeMillis(),
3597               expectedCount, res.size());
3598           toggle = !toggle;
3599         }
3600       }
3601 
3602     } finally {
3603       try {
3604         flushThread.done();
3605         flushThread.join();
3606         flushThread.checkNoError();
3607       } catch (InterruptedException ie) {
3608         LOG.warn("Caught exception when joining with flushThread", ie);
3609       }
3610       HRegion.closeHRegion(this.region);
3611       this.region = null;
3612     }
3613   }
3614 
3615   protected class FlushThread extends Thread {
3616     private volatile boolean done;
3617     private Throwable error = null;
3618 
3619     FlushThread() {
3620       super("FlushThread");
3621     }
3622 
3623     public void done() {
3624       done = true;
3625       synchronized (this) {
3626         interrupt();
3627       }
3628     }
3629 
3630     public void checkNoError() {
3631       if (error != null) {
3632         assertNull(error);
3633       }
3634     }
3635 
3636     @Override
3637     public void run() {
3638       done = false;
3639       while (!done) {
3640         synchronized (this) {
3641           try {
3642             wait();
3643           } catch (InterruptedException ignored) {
3644             if (done) {
3645               break;
3646             }
3647           }
3648         }
3649         try {
3650           region.flush(true);
3651         } catch (IOException e) {
3652           if (!done) {
3653             LOG.error("Error while flushing cache", e);
3654             error = e;
3655           }
3656           break;
3657         } catch (Throwable t) {
3658           LOG.error("Uncaught exception", t);
3659           throw t;
3660         }
3661       }
3662     }
3663 
3664     public void flush() {
3665       synchronized (this) {
3666         notify();
3667       }
3668     }
3669   }
3670 
3671   /**
3672    * Writes very wide records and scans for the latest every time.. Flushes and
3673    * compacts the region every now and then to keep things realistic.
3674    *
3675    * @throws IOException
3676    *           by flush / scan / compaction
3677    * @throws InterruptedException
3678    *           when joining threads
3679    */
3680   @Test
3681   public void testWritesWhileScanning() throws IOException, InterruptedException {
3682     int testCount = 100;
3683     int numRows = 1;
3684     int numFamilies = 10;
3685     int numQualifiers = 100;
3686     int flushInterval = 7;
3687     int compactInterval = 5 * flushInterval;
3688     byte[][] families = new byte[numFamilies][];
3689     for (int i = 0; i < numFamilies; i++) {
3690       families[i] = Bytes.toBytes("family" + i);
3691     }
3692     byte[][] qualifiers = new byte[numQualifiers][];
3693     for (int i = 0; i < numQualifiers; i++) {
3694       qualifiers[i] = Bytes.toBytes("qual" + i);
3695     }
3696 
3697     String method = "testWritesWhileScanning";
3698     this.region = initHRegion(tableName, method, CONF, families);
3699     FlushThread flushThread = new FlushThread();
3700     PutThread putThread = new PutThread(numRows, families, qualifiers);
3701     try {
3702       putThread.start();
3703       putThread.waitForFirstPut();
3704 
3705       flushThread.start();
3706 
3707       Scan scan = new Scan(Bytes.toBytes("row0"), Bytes.toBytes("row1"));
3708 
3709       int expectedCount = numFamilies * numQualifiers;
3710       List<Cell> res = new ArrayList<Cell>();
3711 
3712       long prevTimestamp = 0L;
3713       for (int i = 0; i < testCount; i++) {
3714 
3715         if (i != 0 && i % compactInterval == 0) {
3716           region.compact(true);
3717           for (Store store : region.getStores()) {
3718             store.closeAndArchiveCompactedFiles();
3719           }
3720         }
3721 
3722         if (i != 0 && i % flushInterval == 0) {
3723           flushThread.flush();
3724         }
3725 
3726         boolean previousEmpty = res.isEmpty();
3727         res.clear();
3728         InternalScanner scanner = region.getScanner(scan);
3729         while (scanner.next(res))
3730           ;
3731         if (!res.isEmpty() || !previousEmpty || i > compactInterval) {
3732           assertEquals("i=" + i, expectedCount, res.size());
3733           long timestamp = res.get(0).getTimestamp();
3734           assertTrue("Timestamps were broke: " + timestamp + " prev: " + prevTimestamp,
3735               timestamp >= prevTimestamp);
3736           prevTimestamp = timestamp;
3737         }
3738       }
3739       putThread.done();
3740       region.flush(true);
3741 
3742     } finally {
3743       try {
3744         flushThread.done();
3745         flushThread.join();
3746         flushThread.checkNoError();
3747 
3748         putThread.join();
3749         putThread.checkNoError();
3750       } catch (InterruptedException ie) {
3751         LOG.warn("Caught exception when joining with flushThread", ie);
3752       }
3753 
3754       try {
3755         LOG.info("Before close: " + this.region.getMVCC());
3756         HRegion.closeHRegion(this.region);
3757       } catch (DroppedSnapshotException dse) {
3758         // We could get this on way out because we interrupt the background flusher and it could
3759         // fail anywhere causing a DSE over in the background flusher... only it is not properly
3760         // dealt with so could still be memory hanging out when we get to here -- memory we can't
3761         // flush because the accounting is 'off' since original DSE.
3762       }
3763       this.region = null;
3764     }
3765   }
3766 
3767   protected class PutThread extends Thread {
3768     private boolean done;
3769     private int numPutsFinished = 0;
3770 
3771     private Throwable error = null;
3772     private int numRows;
3773     private byte[][] families;
3774     private byte[][] qualifiers;
3775 
3776     private PutThread(int numRows, byte[][] families, byte[][] qualifiers) {
3777       super("PutThread");
3778       this.numRows = numRows;
3779       this.families = families;
3780       this.qualifiers = qualifiers;
3781     }
3782 
3783     /**
3784      * Block until this thread has put at least one row.
3785      */
3786     public void waitForFirstPut() throws InterruptedException {
3787       // wait until put thread actually puts some data
3788       while (numPutsFinished == 0) {
3789         checkNoError();
3790         Thread.sleep(50);
3791       }
3792     }
3793 
3794     public void done() {
3795       done = true;
3796       synchronized (this) {
3797         interrupt();
3798       }
3799     }
3800 
3801     public void checkNoError() {
3802       if (error != null) {
3803         assertNull(error);
3804       }
3805     }
3806 
3807     @Override
3808     public void run() {
3809       done = false;
3810       while (!done) {
3811         try {
3812           for (int r = 0; r < numRows; r++) {
3813             byte[] row = Bytes.toBytes("row" + r);
3814             Put put = new Put(row);
3815             put.setDurability(Durability.SKIP_WAL);
3816             byte[] value = Bytes.toBytes(String.valueOf(numPutsFinished));
3817             for (byte[] family : families) {
3818               for (byte[] qualifier : qualifiers) {
3819                 put.add(family, qualifier, (long) numPutsFinished, value);
3820               }
3821             }
3822             region.put(put);
3823             numPutsFinished++;
3824             if (numPutsFinished > 0 && numPutsFinished % 47 == 0) {
3825               System.out.println("put iteration = " + numPutsFinished);
3826               Delete delete = new Delete(row, (long) numPutsFinished - 30);
3827               region.delete(delete);
3828             }
3829             numPutsFinished++;
3830           }
3831         } catch (InterruptedIOException e) {
3832           // This is fine. It means we are done, or didn't get the lock on time
3833           LOG.info("Interrupted", e);
3834         } catch (IOException e) {
3835           LOG.error("Error while putting records", e);
3836           error = e;
3837           break;
3838         }
3839       }
3840 
3841     }
3842 
3843   }
3844 
3845   /**
3846    * Writes very wide records and gets the latest row every time.. Flushes and
3847    * compacts the region aggressivly to catch issues.
3848    *
3849    * @throws IOException
3850    *           by flush / scan / compaction
3851    * @throws InterruptedException
3852    *           when joining threads
3853    */
3854   @Test
3855   public void testWritesWhileGetting() throws Exception {
3856     int testCount = 100;
3857     int numRows = 1;
3858     int numFamilies = 10;
3859     int numQualifiers = 100;
3860     int compactInterval = 100;
3861     byte[][] families = new byte[numFamilies][];
3862     for (int i = 0; i < numFamilies; i++) {
3863       families[i] = Bytes.toBytes("family" + i);
3864     }
3865     byte[][] qualifiers = new byte[numQualifiers][];
3866     for (int i = 0; i < numQualifiers; i++) {
3867       qualifiers[i] = Bytes.toBytes("qual" + i);
3868     }
3869 
3870 
3871     String method = "testWritesWhileGetting";
3872     // This test flushes constantly and can cause many files to be created,
3873     // possibly
3874     // extending over the ulimit. Make sure compactions are aggressive in
3875     // reducing
3876     // the number of HFiles created.
3877     Configuration conf = HBaseConfiguration.create(CONF);
3878     conf.setInt("hbase.hstore.compaction.min", 1);
3879     conf.setInt("hbase.hstore.compaction.max", 1000);
3880     this.region = initHRegion(tableName, method, conf, families);
3881     PutThread putThread = null;
3882     MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext(conf);
3883     try {
3884       putThread = new PutThread(numRows, families, qualifiers);
3885       putThread.start();
3886       putThread.waitForFirstPut();
3887 
3888       // Add a thread that flushes as fast as possible
3889       ctx.addThread(new RepeatingTestThread(ctx) {
3890 
3891         @Override
3892         public void doAnAction() throws Exception {
3893           region.flush(true); 
3894           // Compact regularly to avoid creating too many files and exceeding
3895           // the ulimit.
3896           region.compact(false);
3897           for (Store store : region.getStores()) {
3898             store.closeAndArchiveCompactedFiles();
3899           }
3900         }
3901       });
3902       ctx.startThreads();
3903 
3904       Get get = new Get(Bytes.toBytes("row0"));
3905       Result result = null;
3906 
3907       int expectedCount = numFamilies * numQualifiers;
3908 
3909       long prevTimestamp = 0L;
3910       for (int i = 0; i < testCount; i++) {
3911         LOG.info("testWritesWhileGetting verify turn " + i);
3912         boolean previousEmpty = result == null || result.isEmpty();
3913         result = region.get(get);
3914         if (!result.isEmpty() || !previousEmpty || i > compactInterval) {
3915           assertEquals("i=" + i, expectedCount, result.size());
3916           // TODO this was removed, now what dangit?!
3917           // search looking for the qualifier in question?
3918           long timestamp = 0;
3919           for (Cell kv : result.rawCells()) {
3920             if (CellUtil.matchingFamily(kv, families[0])
3921                 && CellUtil.matchingQualifier(kv, qualifiers[0])) {
3922               timestamp = kv.getTimestamp();
3923             }
3924           }
3925           assertTrue(timestamp >= prevTimestamp);
3926           prevTimestamp = timestamp;
3927           Cell previousKV = null;
3928 
3929           for (Cell kv : result.rawCells()) {
3930             byte[] thisValue = CellUtil.cloneValue(kv);
3931             if (previousKV != null) {
3932               if (Bytes.compareTo(CellUtil.cloneValue(previousKV), thisValue) != 0) {
3933                 LOG.warn("These two KV should have the same value." + " Previous KV:" + previousKV
3934                     + "(memStoreTS:" + previousKV.getMvccVersion() + ")" + ", New KV: " + kv
3935                     + "(memStoreTS:" + kv.getMvccVersion() + ")");
3936                 assertEquals(0, Bytes.compareTo(CellUtil.cloneValue(previousKV), thisValue));
3937               }
3938             }
3939             previousKV = kv;
3940           }
3941         }
3942       }
3943     } finally {
3944       if (putThread != null)
3945         putThread.done();
3946 
3947       region.flush(true);
3948 
3949       if (putThread != null) {
3950         putThread.join();
3951         putThread.checkNoError();
3952       }
3953 
3954       ctx.stop();
3955       HRegion.closeHRegion(this.region);
3956       this.region = null;
3957     }
3958   }
3959 
3960   @Test
3961   public void testHolesInMeta() throws Exception {
3962     byte[] family = Bytes.toBytes("family");
3963     this.region = initHRegion(tableName, Bytes.toBytes("x"), Bytes.toBytes("z"), method, CONF,
3964         false, family);
3965     byte[] rowNotServed = Bytes.toBytes("a");
3966     Get g = new Get(rowNotServed);
3967     try {
3968       region.get(g);
3969       fail();
3970     } catch (WrongRegionException x) {
3971       // OK
3972     }
3973     byte[] row = Bytes.toBytes("y");
3974     g = new Get(row);
3975     region.get(g);
3976   }
3977 
3978   @Test
3979   public void testIndexesScanWithOneDeletedRow() throws IOException {
3980     byte[] family = Bytes.toBytes("family");
3981 
3982     // Setting up region
3983     String method = "testIndexesScanWithOneDeletedRow";
3984     this.region = initHRegion(tableName, method, CONF, family);
3985     Put put = new Put(Bytes.toBytes(1L));
3986     put.add(family, qual1, 1L, Bytes.toBytes(1L));
3987     region.put(put);
3988 
3989     region.flush(true);
3990 
3991     Delete delete = new Delete(Bytes.toBytes(1L), 1L);
3992     region.delete(delete);
3993 
3994     put = new Put(Bytes.toBytes(2L));
3995     put.add(family, qual1, 2L, Bytes.toBytes(2L));
3996     region.put(put);
3997 
3998     Scan idxScan = new Scan();
3999     idxScan.addFamily(family);
4000     idxScan.setFilter(new FilterList(FilterList.Operator.MUST_PASS_ALL, Arrays.<Filter> asList(
4001         new SingleColumnValueFilter(family, qual1, CompareOp.GREATER_OR_EQUAL,
4002             new BinaryComparator(Bytes.toBytes(0L))), new SingleColumnValueFilter(family, qual1,
4003             CompareOp.LESS_OR_EQUAL, new BinaryComparator(Bytes.toBytes(3L))))));
4004     InternalScanner scanner = region.getScanner(idxScan);
4005     List<Cell> res = new ArrayList<>();
4006 
4007     while (scanner.next(res)) {
4008       // Ignore res value.
4009     }
4010     assertEquals(1L, res.size());
4011   }
4012 
4013   // ////////////////////////////////////////////////////////////////////////////
4014   // Bloom filter test
4015   // ////////////////////////////////////////////////////////////////////////////
4016   @Test
4017   public void testBloomFilterSize() throws IOException {
4018     byte[] fam1 = Bytes.toBytes("fam1");
4019     byte[] qf1 = Bytes.toBytes("col");
4020     byte[] val1 = Bytes.toBytes("value1");
4021     // Create Table
4022     HColumnDescriptor hcd = new HColumnDescriptor(fam1).setMaxVersions(Integer.MAX_VALUE)
4023         .setBloomFilterType(BloomType.ROWCOL);
4024 
4025     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
4026     htd.addFamily(hcd);
4027     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
4028     this.region = TEST_UTIL.createLocalHRegion(info, htd);
4029     int num_unique_rows = 10;
4030     int duplicate_multiplier = 2;
4031     int num_storefiles = 4;
4032 
4033     int version = 0;
4034     for (int f = 0; f < num_storefiles; f++) {
4035       for (int i = 0; i < duplicate_multiplier; i++) {
4036         for (int j = 0; j < num_unique_rows; j++) {
4037           Put put = new Put(Bytes.toBytes("row" + j));
4038           put.setDurability(Durability.SKIP_WAL);
4039           put.add(fam1, qf1, version++, val1);
4040           region.put(put);
4041         }
4042       }
4043       region.flush(true);
4044     }
4045     // before compaction
4046     HStore store = (HStore) region.getStore(fam1);
4047     Collection<StoreFile> storeFiles = store.getStorefiles();
4048     for (StoreFile storefile : storeFiles) {
4049       StoreFile.Reader reader = storefile.getReader();
4050       reader.loadFileInfo();
4051       reader.loadBloomfilter();
4052       assertEquals(num_unique_rows * duplicate_multiplier, reader.getEntries());
4053       assertEquals(num_unique_rows, reader.getFilterEntries());
4054     }
4055 
4056     region.compact(true);
4057 
4058     // after compaction
4059     storeFiles = store.getStorefiles();
4060     for (StoreFile storefile : storeFiles) {
4061       StoreFile.Reader reader = storefile.getReader();
4062       reader.loadFileInfo();
4063       reader.loadBloomfilter();
4064       assertEquals(num_unique_rows * duplicate_multiplier * num_storefiles, reader.getEntries());
4065       assertEquals(num_unique_rows, reader.getFilterEntries());
4066     }
4067   }
4068 
4069   @Test
4070   public void testAllColumnsWithBloomFilter() throws IOException {
4071     byte[] TABLE = Bytes.toBytes("testAllColumnsWithBloomFilter");
4072     byte[] FAMILY = Bytes.toBytes("family");
4073 
4074     // Create table
4075     HColumnDescriptor hcd = new HColumnDescriptor(FAMILY).setMaxVersions(Integer.MAX_VALUE)
4076         .setBloomFilterType(BloomType.ROWCOL);
4077     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE));
4078     htd.addFamily(hcd);
4079     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
4080     this.region = TEST_UTIL.createLocalHRegion(info, htd);
4081     // For row:0, col:0: insert versions 1 through 5.
4082     byte row[] = Bytes.toBytes("row:" + 0);
4083     byte column[] = Bytes.toBytes("column:" + 0);
4084     Put put = new Put(row);
4085     put.setDurability(Durability.SKIP_WAL);
4086     for (long idx = 1; idx <= 4; idx++) {
4087       put.add(FAMILY, column, idx, Bytes.toBytes("value-version-" + idx));
4088     }
4089     region.put(put);
4090 
4091     // Flush
4092     region.flush(true);
4093 
4094     // Get rows
4095     Get get = new Get(row);
4096     get.setMaxVersions();
4097     Cell[] kvs = region.get(get).rawCells();
4098 
4099     // Check if rows are correct
4100     assertEquals(4, kvs.length);
4101     checkOneCell(kvs[0], FAMILY, 0, 0, 4);
4102     checkOneCell(kvs[1], FAMILY, 0, 0, 3);
4103     checkOneCell(kvs[2], FAMILY, 0, 0, 2);
4104     checkOneCell(kvs[3], FAMILY, 0, 0, 1);
4105   }
4106 
4107   /**
4108    * Testcase to cover bug-fix for HBASE-2823 Ensures correct delete when
4109    * issuing delete row on columns with bloom filter set to row+col
4110    * (BloomType.ROWCOL)
4111    */
4112   @Test
4113   public void testDeleteRowWithBloomFilter() throws IOException {
4114     byte[] familyName = Bytes.toBytes("familyName");
4115 
4116     // Create Table
4117     HColumnDescriptor hcd = new HColumnDescriptor(familyName).setMaxVersions(Integer.MAX_VALUE)
4118         .setBloomFilterType(BloomType.ROWCOL);
4119 
4120     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
4121     htd.addFamily(hcd);
4122     HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
4123     this.region = TEST_UTIL.createLocalHRegion(info, htd);
4124     // Insert some data
4125     byte row[] = Bytes.toBytes("row1");
4126     byte col[] = Bytes.toBytes("col1");
4127 
4128     Put put = new Put(row);
4129     put.add(familyName, col, 1, Bytes.toBytes("SomeRandomValue"));
4130     region.put(put);
4131     region.flush(true);
4132 
4133     Delete del = new Delete(row);
4134     region.delete(del);
4135     region.flush(true);
4136 
4137     // Get remaining rows (should have none)
4138     Get get = new Get(row);
4139     get.addColumn(familyName, col);
4140 
4141     Cell[] keyValues = region.get(get).rawCells();
4142     assertEquals(0,keyValues.length);
4143   }
4144 
4145   @Test
4146   public void testgetHDFSBlocksDistribution() throws Exception {
4147     HBaseTestingUtility htu = new HBaseTestingUtility();
4148     // Why do we set the block size in this test?  If we set it smaller than the kvs, then we'll
4149     // break up the file in to more pieces that can be distributed across the three nodes and we
4150     // won't be able to have the condition this test asserts; that at least one node has
4151     // a copy of all replicas -- if small block size, then blocks are spread evenly across the
4152     // the three nodes.  hfilev3 with tags seems to put us over the block size.  St.Ack.
4153     // final int DEFAULT_BLOCK_SIZE = 1024;
4154     // htu.getConfiguration().setLong("dfs.blocksize", DEFAULT_BLOCK_SIZE);
4155     htu.getConfiguration().setInt("dfs.replication", 2);
4156 
4157     // set up a cluster with 3 nodes
4158     MiniHBaseCluster cluster = null;
4159     String dataNodeHosts[] = new String[] { "host1", "host2", "host3" };
4160     int regionServersCount = 3;
4161 
4162     try {
4163       cluster = htu.startMiniCluster(1, regionServersCount, dataNodeHosts);
4164       byte[][] families = { fam1, fam2 };
4165       Table ht = htu.createTable(Bytes.toBytes(this.getName()), families);
4166 
4167       // Setting up region
4168       byte row[] = Bytes.toBytes("row1");
4169       byte col[] = Bytes.toBytes("col1");
4170 
4171       Put put = new Put(row);
4172       put.add(fam1, col, 1, Bytes.toBytes("test1"));
4173       put.add(fam2, col, 1, Bytes.toBytes("test2"));
4174       ht.put(put);
4175 
4176       HRegion firstRegion = htu.getHBaseCluster().getRegions(TableName.valueOf(this.getName()))
4177           .get(0);
4178       firstRegion.flush(true);
4179       HDFSBlocksDistribution blocksDistribution1 = firstRegion.getHDFSBlocksDistribution();
4180 
4181       // Given the default replication factor is 2 and we have 2 HFiles,
4182       // we will have total of 4 replica of blocks on 3 datanodes; thus there
4183       // must be at least one host that have replica for 2 HFiles. That host's
4184       // weight will be equal to the unique block weight.
4185       long uniqueBlocksWeight1 = blocksDistribution1.getUniqueBlocksTotalWeight();
4186       StringBuilder sb = new StringBuilder();
4187       for (String host: blocksDistribution1.getTopHosts()) {
4188         if (sb.length() > 0) sb.append(", ");
4189         sb.append(host);
4190         sb.append("=");
4191         sb.append(blocksDistribution1.getWeight(host));
4192       }
4193 
4194       String topHost = blocksDistribution1.getTopHosts().get(0);
4195       long topHostWeight = blocksDistribution1.getWeight(topHost);
4196       String msg = "uniqueBlocksWeight=" + uniqueBlocksWeight1 + ", topHostWeight=" +
4197         topHostWeight + ", topHost=" + topHost + "; " + sb.toString();
4198       LOG.info(msg);
4199       assertTrue(msg, uniqueBlocksWeight1 == topHostWeight);
4200 
4201       // use the static method to compute the value, it should be the same.
4202       // static method is used by load balancer or other components
4203       HDFSBlocksDistribution blocksDistribution2 = HRegion.computeHDFSBlocksDistribution(
4204           htu.getConfiguration(), firstRegion.getTableDesc(), firstRegion.getRegionInfo());
4205       long uniqueBlocksWeight2 = blocksDistribution2.getUniqueBlocksTotalWeight();
4206 
4207       assertTrue(uniqueBlocksWeight1 == uniqueBlocksWeight2);
4208 
4209       ht.close();
4210     } finally {
4211       if (cluster != null) {
4212         htu.shutdownMiniCluster();
4213       }
4214     }
4215   }
4216 
4217   /**
4218    * Testcase to check state of region initialization task set to ABORTED or not
4219    * if any exceptions during initialization
4220    *
4221    * @throws Exception
4222    */
4223   @Test
4224   public void testStatusSettingToAbortIfAnyExceptionDuringRegionInitilization() throws Exception {
4225     TableName tableName = TableName.valueOf(name.getMethodName());
4226     HRegionInfo info = null;
4227     try {
4228       FileSystem fs = Mockito.mock(FileSystem.class);
4229       Mockito.when(fs.exists((Path) Mockito.anyObject())).thenThrow(new IOException());
4230       HTableDescriptor htd = new HTableDescriptor(tableName);
4231       htd.addFamily(new HColumnDescriptor("cf"));
4232       info = new HRegionInfo(htd.getTableName(), HConstants.EMPTY_BYTE_ARRAY,
4233           HConstants.EMPTY_BYTE_ARRAY, false);
4234       Path path = new Path(dir + "testStatusSettingToAbortIfAnyExceptionDuringRegionInitilization");
4235       region = HRegion.newHRegion(path, null, fs, CONF, info, htd, null);
4236       // region initialization throws IOException and set task state to ABORTED.
4237       region.initialize();
4238       fail("Region initialization should fail due to IOException");
4239     } catch (IOException io) {
4240       List<MonitoredTask> tasks = TaskMonitor.get().getTasks();
4241       for (MonitoredTask monitoredTask : tasks) {
4242         if (!(monitoredTask instanceof MonitoredRPCHandler)
4243             && monitoredTask.getDescription().contains(region.toString())) {
4244           assertTrue("Region state should be ABORTED.",
4245               monitoredTask.getState().equals(MonitoredTask.State.ABORTED));
4246           break;
4247         }
4248       }
4249     }
4250   }
4251 
4252   /**
4253    * Verifies that the .regioninfo file is written on region creation and that
4254    * is recreated if missing during region opening.
4255    */
4256   @Test
4257   public void testRegionInfoFileCreation() throws IOException {
4258     Path rootDir = new Path(dir + "testRegionInfoFileCreation");
4259 
4260     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testtb"));
4261     htd.addFamily(new HColumnDescriptor("cf"));
4262 
4263     HRegionInfo hri = new HRegionInfo(htd.getTableName());
4264 
4265     // Create a region and skip the initialization (like CreateTableHandler)
4266     region = HRegion.createHRegion(hri, rootDir, CONF, htd, null, false, true);
4267 //    HRegion region = TEST_UTIL.createLocalHRegion(hri, htd);
4268     Path regionDir = region.getRegionFileSystem().getRegionDir();
4269     FileSystem fs = region.getRegionFileSystem().getFileSystem();
4270     HRegion.closeHRegion(region);
4271 
4272     Path regionInfoFile = new Path(regionDir, HRegionFileSystem.REGION_INFO_FILE);
4273 
4274     // Verify that the .regioninfo file is present
4275     assertTrue(HRegionFileSystem.REGION_INFO_FILE + " should be present in the region dir",
4276         fs.exists(regionInfoFile));
4277 
4278     // Try to open the region
4279     region = HRegion.openHRegion(rootDir, hri, htd, null, CONF);
4280     assertEquals(regionDir, region.getRegionFileSystem().getRegionDir());
4281     HRegion.closeHRegion(region);
4282 
4283     // Verify that the .regioninfo file is still there
4284     assertTrue(HRegionFileSystem.REGION_INFO_FILE + " should be present in the region dir",
4285         fs.exists(regionInfoFile));
4286 
4287     // Remove the .regioninfo file and verify is recreated on region open
4288     fs.delete(regionInfoFile, true);
4289     assertFalse(HRegionFileSystem.REGION_INFO_FILE + " should be removed from the region dir",
4290         fs.exists(regionInfoFile));
4291 
4292     region = HRegion.openHRegion(rootDir, hri, htd, null, CONF);
4293 //    region = TEST_UTIL.openHRegion(hri, htd);
4294     assertEquals(regionDir, region.getRegionFileSystem().getRegionDir());
4295     HRegion.closeHRegion(region);
4296 
4297     // Verify that the .regioninfo file is still there
4298     assertTrue(HRegionFileSystem.REGION_INFO_FILE + " should be present in the region dir",
4299         fs.exists(new Path(regionDir, HRegionFileSystem.REGION_INFO_FILE)));
4300 
4301     region = null;
4302   }
4303 
4304   /**
4305    * TestCase for increment
4306    */
4307   private static class Incrementer implements Runnable {
4308     private HRegion region;
4309     private final static byte[] incRow = Bytes.toBytes("incRow");
4310     private final static byte[] family = Bytes.toBytes("family");
4311     private final static byte[] qualifier = Bytes.toBytes("qualifier");
4312     private final static long ONE = 1l;
4313     private int incCounter;
4314 
4315     public Incrementer(HRegion region, int incCounter) {
4316       this.region = region;
4317       this.incCounter = incCounter;
4318     }
4319 
4320     @Override
4321     public void run() {
4322       int count = 0;
4323       while (count < incCounter) {
4324         Increment inc = new Increment(incRow);
4325         inc.addColumn(family, qualifier, ONE);
4326         count++;
4327         try {
4328           region.increment(inc);
4329         } catch (IOException e) {
4330           e.printStackTrace();
4331           break;
4332         }
4333       }
4334     }
4335   }
4336 
4337   /**
4338    * Test case to check increment function with memstore flushing
4339    * @throws Exception
4340    */
4341   @Test
4342   public void testParallelIncrementWithMemStoreFlush() throws Exception {
4343     byte[] family = Incrementer.family;
4344     this.region = initHRegion(tableName, method, CONF, family);
4345     final HRegion region = this.region;
4346     final AtomicBoolean incrementDone = new AtomicBoolean(false);
4347     Runnable flusher = new Runnable() {
4348       @Override
4349       public void run() {
4350         while (!incrementDone.get()) {
4351           try {
4352             region.flush(true);
4353           } catch (Exception e) {
4354             e.printStackTrace();
4355           }
4356         }
4357       }
4358     };
4359 
4360     // after all increment finished, the row will increment to 20*100 = 2000
4361     int threadNum = 20;
4362     int incCounter = 100;
4363     long expected = threadNum * incCounter;
4364     Thread[] incrementers = new Thread[threadNum];
4365     Thread flushThread = new Thread(flusher);
4366     for (int i = 0; i < threadNum; i++) {
4367       incrementers[i] = new Thread(new Incrementer(this.region, incCounter));
4368       incrementers[i].start();
4369     }
4370     flushThread.start();
4371     for (int i = 0; i < threadNum; i++) {
4372       incrementers[i].join();
4373     }
4374 
4375     incrementDone.set(true);
4376     flushThread.join();
4377 
4378     Get get = new Get(Incrementer.incRow);
4379     get.addColumn(Incrementer.family, Incrementer.qualifier);
4380     get.setMaxVersions(1);
4381     Result res = this.region.get(get);
4382     List<Cell> kvs = res.getColumnCells(Incrementer.family, Incrementer.qualifier);
4383 
4384     // we just got the latest version
4385     assertEquals(kvs.size(), 1);
4386     Cell kv = kvs.get(0);
4387     assertEquals(expected, Bytes.toLong(kv.getValueArray(), kv.getValueOffset()));
4388   }
4389 
4390   /**
4391    * TestCase for append
4392    */
4393   private static class Appender implements Runnable {
4394     private HRegion region;
4395     private final static byte[] appendRow = Bytes.toBytes("appendRow");
4396     private final static byte[] family = Bytes.toBytes("family");
4397     private final static byte[] qualifier = Bytes.toBytes("qualifier");
4398     private final static byte[] CHAR = Bytes.toBytes("a");
4399     private int appendCounter;
4400 
4401     public Appender(HRegion region, int appendCounter) {
4402       this.region = region;
4403       this.appendCounter = appendCounter;
4404     }
4405 
4406     @Override
4407     public void run() {
4408       int count = 0;
4409       while (count < appendCounter) {
4410         Append app = new Append(appendRow);
4411         app.add(family, qualifier, CHAR);
4412         count++;
4413         try {
4414           region.append(app);
4415         } catch (IOException e) {
4416           e.printStackTrace();
4417           break;
4418         }
4419       }
4420     }
4421   }
4422 
4423   /**
4424    * Test case to check append function with memstore flushing
4425    * @throws Exception
4426    */
4427   @Test
4428   public void testParallelAppendWithMemStoreFlush() throws Exception {
4429     byte[] family = Appender.family;
4430     this.region = initHRegion(tableName, method, CONF, family);
4431     final HRegion region = this.region;
4432     final AtomicBoolean appendDone = new AtomicBoolean(false);
4433     Runnable flusher = new Runnable() {
4434       @Override
4435       public void run() {
4436         while (!appendDone.get()) {
4437           try {
4438             region.flush(true);
4439           } catch (Exception e) {
4440             e.printStackTrace();
4441           }
4442         }
4443       }
4444     };
4445 
4446     // after all append finished, the value will append to threadNum *
4447     // appendCounter Appender.CHAR
4448     int threadNum = 20;
4449     int appendCounter = 100;
4450     byte[] expected = new byte[threadNum * appendCounter];
4451     for (int i = 0; i < threadNum * appendCounter; i++) {
4452       System.arraycopy(Appender.CHAR, 0, expected, i, 1);
4453     }
4454     Thread[] appenders = new Thread[threadNum];
4455     Thread flushThread = new Thread(flusher);
4456     for (int i = 0; i < threadNum; i++) {
4457       appenders[i] = new Thread(new Appender(this.region, appendCounter));
4458       appenders[i].start();
4459     }
4460     flushThread.start();
4461     for (int i = 0; i < threadNum; i++) {
4462       appenders[i].join();
4463     }
4464 
4465     appendDone.set(true);
4466     flushThread.join();
4467 
4468     Get get = new Get(Appender.appendRow);
4469     get.addColumn(Appender.family, Appender.qualifier);
4470     get.setMaxVersions(1);
4471     Result res = this.region.get(get);
4472     List<Cell> kvs = res.getColumnCells(Appender.family, Appender.qualifier);
4473 
4474     // we just got the latest version
4475     assertEquals(kvs.size(), 1);
4476     Cell kv = kvs.get(0);
4477     byte[] appendResult = new byte[kv.getValueLength()];
4478     System.arraycopy(kv.getValueArray(), kv.getValueOffset(), appendResult, 0, kv.getValueLength());
4479     assertArrayEquals(expected, appendResult);
4480   }
4481 
4482   /**
4483    * Test case to check put function with memstore flushing for same row, same ts
4484    * @throws Exception
4485    */
4486   @Test
4487   public void testPutWithMemStoreFlush() throws Exception {
4488     byte[] family = Bytes.toBytes("family");
4489     ;
4490     byte[] qualifier = Bytes.toBytes("qualifier");
4491     byte[] row = Bytes.toBytes("putRow");
4492     byte[] value = null;
4493     this.region = initHRegion(tableName, method, CONF, family);
4494     Put put = null;
4495     Get get = null;
4496     List<Cell> kvs = null;
4497     Result res = null;
4498 
4499     put = new Put(row);
4500     value = Bytes.toBytes("value0");
4501     put.add(family, qualifier, 1234567l, value);
4502     region.put(put);
4503     get = new Get(row);
4504     get.addColumn(family, qualifier);
4505     get.setMaxVersions();
4506     res = this.region.get(get);
4507     kvs = res.getColumnCells(family, qualifier);
4508     assertEquals(1, kvs.size());
4509     assertArrayEquals(Bytes.toBytes("value0"), CellUtil.cloneValue(kvs.get(0)));
4510 
4511     region.flush(true);
4512     get = new Get(row);
4513     get.addColumn(family, qualifier);
4514     get.setMaxVersions();
4515     res = this.region.get(get);
4516     kvs = res.getColumnCells(family, qualifier);
4517     assertEquals(1, kvs.size());
4518     assertArrayEquals(Bytes.toBytes("value0"), CellUtil.cloneValue(kvs.get(0)));
4519 
4520     put = new Put(row);
4521     value = Bytes.toBytes("value1");
4522     put.add(family, qualifier, 1234567l, value);
4523     region.put(put);
4524     get = new Get(row);
4525     get.addColumn(family, qualifier);
4526     get.setMaxVersions();
4527     res = this.region.get(get);
4528     kvs = res.getColumnCells(family, qualifier);
4529     assertEquals(1, kvs.size());
4530     assertArrayEquals(Bytes.toBytes("value1"), CellUtil.cloneValue(kvs.get(0)));
4531 
4532     region.flush(true);
4533     get = new Get(row);
4534     get.addColumn(family, qualifier);
4535     get.setMaxVersions();
4536     res = this.region.get(get);
4537     kvs = res.getColumnCells(family, qualifier);
4538     assertEquals(1, kvs.size());
4539     assertArrayEquals(Bytes.toBytes("value1"), CellUtil.cloneValue(kvs.get(0)));
4540   }
4541 
4542   @Test
4543   public void testDurability() throws Exception {
4544     String method = "testDurability";
4545     // there are 5 x 5 cases:
4546     // table durability(SYNC,FSYNC,ASYC,SKIP,USE_DEFAULT) x mutation
4547     // durability(SYNC,FSYNC,ASYC,SKIP,USE_DEFAULT)
4548 
4549     // expected cases for append and sync wal
4550     durabilityTest(method, Durability.SYNC_WAL, Durability.SYNC_WAL, 0, true, true, false);
4551     durabilityTest(method, Durability.SYNC_WAL, Durability.FSYNC_WAL, 0, true, true, false);
4552     durabilityTest(method, Durability.SYNC_WAL, Durability.USE_DEFAULT, 0, true, true, false);
4553 
4554     durabilityTest(method, Durability.FSYNC_WAL, Durability.SYNC_WAL, 0, true, true, false);
4555     durabilityTest(method, Durability.FSYNC_WAL, Durability.FSYNC_WAL, 0, true, true, false);
4556     durabilityTest(method, Durability.FSYNC_WAL, Durability.USE_DEFAULT, 0, true, true, false);
4557 
4558     durabilityTest(method, Durability.ASYNC_WAL, Durability.SYNC_WAL, 0, true, true, false);
4559     durabilityTest(method, Durability.ASYNC_WAL, Durability.FSYNC_WAL, 0, true, true, false);
4560 
4561     durabilityTest(method, Durability.SKIP_WAL, Durability.SYNC_WAL, 0, true, true, false);
4562     durabilityTest(method, Durability.SKIP_WAL, Durability.FSYNC_WAL, 0, true, true, false);
4563 
4564     durabilityTest(method, Durability.USE_DEFAULT, Durability.SYNC_WAL, 0, true, true, false);
4565     durabilityTest(method, Durability.USE_DEFAULT, Durability.FSYNC_WAL, 0, true, true, false);
4566     durabilityTest(method, Durability.USE_DEFAULT, Durability.USE_DEFAULT, 0, true, true, false);
4567 
4568     // expected cases for async wal
4569     durabilityTest(method, Durability.SYNC_WAL, Durability.ASYNC_WAL, 0, true, false, false);
4570     durabilityTest(method, Durability.FSYNC_WAL, Durability.ASYNC_WAL, 0, true, false, false);
4571     durabilityTest(method, Durability.ASYNC_WAL, Durability.ASYNC_WAL, 0, true, false, false);
4572     durabilityTest(method, Durability.SKIP_WAL, Durability.ASYNC_WAL, 0, true, false, false);
4573     durabilityTest(method, Durability.USE_DEFAULT, Durability.ASYNC_WAL, 0, true, false, false);
4574     durabilityTest(method, Durability.ASYNC_WAL, Durability.USE_DEFAULT, 0, true, false, false);
4575 
4576     durabilityTest(method, Durability.SYNC_WAL, Durability.ASYNC_WAL, 5000, true, false, true);
4577     durabilityTest(method, Durability.FSYNC_WAL, Durability.ASYNC_WAL, 5000, true, false, true);
4578     durabilityTest(method, Durability.ASYNC_WAL, Durability.ASYNC_WAL, 5000, true, false, true);
4579     durabilityTest(method, Durability.SKIP_WAL, Durability.ASYNC_WAL, 5000, true, false, true);
4580     durabilityTest(method, Durability.USE_DEFAULT, Durability.ASYNC_WAL, 5000, true, false, true);
4581     durabilityTest(method, Durability.ASYNC_WAL, Durability.USE_DEFAULT, 5000, true, false, true);
4582 
4583     // expect skip wal cases
4584     durabilityTest(method, Durability.SYNC_WAL, Durability.SKIP_WAL, 0, true, false, false);
4585     durabilityTest(method, Durability.FSYNC_WAL, Durability.SKIP_WAL, 0, true, false, false);
4586     durabilityTest(method, Durability.ASYNC_WAL, Durability.SKIP_WAL, 0, true, false, false);
4587     durabilityTest(method, Durability.SKIP_WAL, Durability.SKIP_WAL, 0, true, false, false);
4588     durabilityTest(method, Durability.USE_DEFAULT, Durability.SKIP_WAL, 0, true, false, false);
4589     durabilityTest(method, Durability.SKIP_WAL, Durability.USE_DEFAULT, 0, true, false, false);
4590 
4591   }
4592 
4593   private void durabilityTest(String method, Durability tableDurability,
4594       Durability mutationDurability, long timeout, boolean expectAppend, final boolean expectSync,
4595       final boolean expectSyncFromLogSyncer) throws Exception {
4596     Configuration conf = HBaseConfiguration.create(CONF);
4597     method = method + "_" + tableDurability.name() + "_" + mutationDurability.name();
4598     TableName tableName = TableName.valueOf(method);
4599     byte[] family = Bytes.toBytes("family");
4600     Path logDir = new Path(new Path(dir + method), "log");
4601     final Configuration walConf = new Configuration(conf);
4602     FSUtils.setRootDir(walConf, logDir);
4603     final WALFactory wals = new WALFactory(walConf, null, UUID.randomUUID().toString());
4604     final WAL wal = spy(wals.getWAL(tableName.getName(), tableName.getNamespace()));
4605     this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW,
4606         HConstants.EMPTY_END_ROW, method, conf, false, tableDurability, wal,
4607         new byte[][] { family });
4608 
4609     Put put = new Put(Bytes.toBytes("r1"));
4610     put.add(family, Bytes.toBytes("q1"), Bytes.toBytes("v1"));
4611     put.setDurability(mutationDurability);
4612     region.put(put);
4613 
4614     //verify append called or not
4615     verify(wal, expectAppend ? times(1) : never())
4616       .append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any(),
4617           (WALEdit)any(), Mockito.anyBoolean());
4618 
4619     // verify sync called or not
4620     if (expectSync || expectSyncFromLogSyncer) {
4621       TEST_UTIL.waitFor(timeout, new Waiter.Predicate<Exception>() {
4622         @Override
4623         public boolean evaluate() throws Exception {
4624           try {
4625             if (expectSync) {
4626               verify(wal, times(1)).sync(anyLong()); // Hregion calls this one
4627             } else if (expectSyncFromLogSyncer) {
4628               verify(wal, times(1)).sync(); // wal syncer calls this one
4629             }
4630           } catch (Throwable ignore) {
4631           }
4632           return true;
4633         }
4634       });
4635     } else {
4636       //verify(wal, never()).sync(anyLong());
4637       verify(wal, never()).sync();
4638     }
4639 
4640     HRegion.closeHRegion(this.region);
4641     this.region = null;
4642   }
4643 
4644   @Test
4645   public void testRegionReplicaSecondary() throws IOException {
4646     // create a primary region, load some data and flush
4647     // create a secondary region, and do a get against that
4648     Path rootDir = new Path(dir + "testRegionReplicaSecondary");
4649     FSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootDir);
4650 
4651     byte[][] families = new byte[][] {
4652         Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3")
4653     };
4654     byte[] cq = Bytes.toBytes("cq");
4655     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testRegionReplicaSecondary"));
4656     for (byte[] family : families) {
4657       htd.addFamily(new HColumnDescriptor(family));
4658     }
4659 
4660     long time = System.currentTimeMillis();
4661     HRegionInfo primaryHri = new HRegionInfo(htd.getTableName(),
4662       HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
4663       false, time, 0);
4664     HRegionInfo secondaryHri = new HRegionInfo(htd.getTableName(),
4665       HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
4666       false, time, 1);
4667 
4668     HRegion primaryRegion = null, secondaryRegion = null;
4669 
4670     try {
4671       primaryRegion = HRegion.createHRegion(primaryHri,
4672         rootDir, TEST_UTIL.getConfiguration(), htd);
4673 
4674       // load some data
4675       putData(primaryRegion, 0, 1000, cq, families);
4676 
4677       // flush region
4678       primaryRegion.flush(true);
4679 
4680       // open secondary region
4681       secondaryRegion = HRegion.openHRegion(rootDir, secondaryHri, htd, null, CONF);
4682 
4683       verifyData(secondaryRegion, 0, 1000, cq, families);
4684     } finally {
4685       if (primaryRegion != null) {
4686         HRegion.closeHRegion(primaryRegion);
4687       }
4688       if (secondaryRegion != null) {
4689         HRegion.closeHRegion(secondaryRegion);
4690       }
4691     }
4692   }
4693 
4694   @Test
4695   public void testRegionReplicaSecondaryIsReadOnly() throws IOException {
4696     // create a primary region, load some data and flush
4697     // create a secondary region, and do a put against that
4698     Path rootDir = new Path(dir + "testRegionReplicaSecondary");
4699     FSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootDir);
4700 
4701     byte[][] families = new byte[][] {
4702         Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3")
4703     };
4704     byte[] cq = Bytes.toBytes("cq");
4705     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testRegionReplicaSecondary"));
4706     for (byte[] family : families) {
4707       htd.addFamily(new HColumnDescriptor(family));
4708     }
4709 
4710     long time = System.currentTimeMillis();
4711     HRegionInfo primaryHri = new HRegionInfo(htd.getTableName(),
4712       HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
4713       false, time, 0);
4714     HRegionInfo secondaryHri = new HRegionInfo(htd.getTableName(),
4715       HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
4716       false, time, 1);
4717 
4718     HRegion primaryRegion = null, secondaryRegion = null;
4719 
4720     try {
4721       primaryRegion = HRegion.createHRegion(primaryHri,
4722         rootDir, TEST_UTIL.getConfiguration(), htd);
4723 
4724       // load some data
4725       putData(primaryRegion, 0, 1000, cq, families);
4726 
4727       // flush region
4728       primaryRegion.flush(true);
4729 
4730       // open secondary region
4731       secondaryRegion = HRegion.openHRegion(rootDir, secondaryHri, htd, null, CONF);
4732 
4733       try {
4734         putData(secondaryRegion, 0, 1000, cq, families);
4735         fail("Should have thrown exception");
4736       } catch (IOException ex) {
4737         // expected
4738       }
4739     } finally {
4740       if (primaryRegion != null) {
4741         HRegion.closeHRegion(primaryRegion);
4742       }
4743       if (secondaryRegion != null) {
4744         HRegion.closeHRegion(secondaryRegion);
4745       }
4746     }
4747   }
4748 
4749   static WALFactory createWALFactory(Configuration conf, Path rootDir) throws IOException {
4750     Configuration confForWAL = new Configuration(conf);
4751     confForWAL.set(HConstants.HBASE_DIR, rootDir.toString());
4752     return new WALFactory(confForWAL,
4753         Collections.<WALActionsListener>singletonList(new MetricsWAL()),
4754         "hregion-" + RandomStringUtils.randomNumeric(8));
4755   }
4756 
4757   @Test
4758   public void testCompactionFromPrimary() throws IOException {
4759     Path rootDir = new Path(dir + "testRegionReplicaSecondary");
4760     FSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootDir);
4761 
4762     byte[][] families = new byte[][] {
4763         Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3")
4764     };
4765     byte[] cq = Bytes.toBytes("cq");
4766     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testRegionReplicaSecondary"));
4767     for (byte[] family : families) {
4768       htd.addFamily(new HColumnDescriptor(family));
4769     }
4770 
4771     long time = System.currentTimeMillis();
4772     HRegionInfo primaryHri = new HRegionInfo(htd.getTableName(),
4773       HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
4774       false, time, 0);
4775     HRegionInfo secondaryHri = new HRegionInfo(htd.getTableName(),
4776       HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
4777       false, time, 1);
4778 
4779     HRegion primaryRegion = null, secondaryRegion = null;
4780 
4781     try {
4782       primaryRegion = HRegion.createHRegion(primaryHri,
4783         rootDir, TEST_UTIL.getConfiguration(), htd);
4784 
4785       // load some data
4786       putData(primaryRegion, 0, 1000, cq, families);
4787 
4788       // flush region
4789       primaryRegion.flush(true);
4790 
4791       // open secondary region
4792       secondaryRegion = HRegion.openHRegion(rootDir, secondaryHri, htd, null, CONF);
4793 
4794       // move the file of the primary region to the archive, simulating a compaction
4795       Collection<StoreFile> storeFiles = primaryRegion.getStore(families[0]).getStorefiles();
4796       primaryRegion.getRegionFileSystem().removeStoreFiles(Bytes.toString(families[0]), storeFiles);
4797       Collection<StoreFileInfo> storeFileInfos = primaryRegion.getRegionFileSystem().getStoreFiles(families[0]);
4798       Assert.assertTrue(storeFileInfos == null || storeFileInfos.size() == 0);
4799 
4800       verifyData(secondaryRegion, 0, 1000, cq, families);
4801     } finally {
4802       if (primaryRegion != null) {
4803         HRegion.closeHRegion(primaryRegion);
4804       }
4805       if (secondaryRegion != null) {
4806         HRegion.closeHRegion(secondaryRegion);
4807       }
4808     }
4809   }
4810 
4811   private void putData(int startRow, int numRows, byte[] qf, byte[]... families) throws IOException {
4812     putData(this.region, startRow, numRows, qf, families);
4813   }
4814 
4815   private void putData(HRegion region,
4816       int startRow, int numRows, byte[] qf, byte[]... families) throws IOException {
4817     putData(region, Durability.SKIP_WAL, startRow, numRows, qf, families);
4818   }
4819 
4820   static void putData(HRegion region, Durability durability,
4821       int startRow, int numRows, byte[] qf, byte[]... families) throws IOException {
4822     for (int i = startRow; i < startRow + numRows; i++) {
4823       Put put = new Put(Bytes.toBytes("" + i));
4824       put.setDurability(durability);
4825       for (byte[] family : families) {
4826         put.add(family, qf, null);
4827       }
4828       region.put(put);
4829     }
4830   }
4831 
4832   static void verifyData(HRegion newReg, int startRow, int numRows, byte[] qf, byte[]... families)
4833       throws IOException {
4834     for (int i = startRow; i < startRow + numRows; i++) {
4835       byte[] row = Bytes.toBytes("" + i);
4836       Get get = new Get(row);
4837       for (byte[] family : families) {
4838         get.addColumn(family, qf);
4839       }
4840       Result result = newReg.get(get);
4841       Cell[] raw = result.rawCells();
4842       assertEquals(families.length, result.size());
4843       for (int j = 0; j < families.length; j++) {
4844         assertTrue(CellUtil.matchingRow(raw[j], row));
4845         assertTrue(CellUtil.matchingFamily(raw[j], families[j]));
4846         assertTrue(CellUtil.matchingQualifier(raw[j], qf));
4847       }
4848     }
4849   }
4850 
4851   static void assertGet(final HRegion r, final byte[] family, final byte[] k) throws IOException {
4852     // Now I have k, get values out and assert they are as expected.
4853     Get get = new Get(k).addFamily(family).setMaxVersions();
4854     Cell[] results = r.get(get).rawCells();
4855     for (int j = 0; j < results.length; j++) {
4856       byte[] tmp = CellUtil.cloneValue(results[j]);
4857       // Row should be equal to value every time.
4858       assertTrue(Bytes.equals(k, tmp));
4859     }
4860   }
4861 
4862   /*
4863    * Assert first value in the passed region is <code>firstValue</code>.
4864    *
4865    * @param r
4866    *
4867    * @param fs
4868    *
4869    * @param firstValue
4870    *
4871    * @throws IOException
4872    */
4873   private void assertScan(final HRegion r, final byte[] fs, final byte[] firstValue)
4874       throws IOException {
4875     byte[][] families = { fs };
4876     Scan scan = new Scan();
4877     for (int i = 0; i < families.length; i++)
4878       scan.addFamily(families[i]);
4879     InternalScanner s = r.getScanner(scan);
4880     try {
4881       List<Cell> curVals = new ArrayList<Cell>();
4882       boolean first = true;
4883       OUTER_LOOP: while (s.next(curVals)) {
4884         for (Cell kv : curVals) {
4885           byte[] val = CellUtil.cloneValue(kv);
4886           byte[] curval = val;
4887           if (first) {
4888             first = false;
4889             assertTrue(Bytes.compareTo(curval, firstValue) == 0);
4890           } else {
4891             // Not asserting anything. Might as well break.
4892             break OUTER_LOOP;
4893           }
4894         }
4895       }
4896     } finally {
4897       s.close();
4898     }
4899   }
4900 
4901   /**
4902    * Test that we get the expected flush results back
4903    * @throws IOException
4904    */
4905   @Test
4906   public void testFlushResult() throws IOException {
4907     String method = name.getMethodName();
4908     byte[] tableName = Bytes.toBytes(method);
4909     byte[] family = Bytes.toBytes("family");
4910 
4911     this.region = initHRegion(tableName, method, family);
4912 
4913     // empty memstore, flush doesn't run
4914     HRegion.FlushResult fr = region.flush(true);
4915     assertFalse(fr.isFlushSucceeded());
4916     assertFalse(fr.isCompactionNeeded());
4917 
4918     // Flush enough files to get up to the threshold, doesn't need compactions
4919     for (int i = 0; i < 2; i++) {
4920       Put put = new Put(tableName).add(family, family, tableName);
4921       region.put(put);
4922       fr = region.flush(true);
4923       assertTrue(fr.isFlushSucceeded());
4924       assertFalse(fr.isCompactionNeeded());
4925     }
4926 
4927     // Two flushes after the threshold, compactions are needed
4928     for (int i = 0; i < 2; i++) {
4929       Put put = new Put(tableName).add(family, family, tableName);
4930       region.put(put);
4931       fr = region.flush(true);
4932       assertTrue(fr.isFlushSucceeded());
4933       assertTrue(fr.isCompactionNeeded());
4934     }
4935   }
4936 
4937   private Configuration initSplit() {
4938     // Always compact if there is more than one store file.
4939     CONF.setInt("hbase.hstore.compactionThreshold", 2);
4940 
4941     // Make lease timeout longer, lease checks less frequent
4942     CONF.setInt("hbase.master.lease.thread.wakefrequency", 5 * 1000);
4943 
4944     CONF.setInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, 10 * 1000);
4945 
4946     // Increase the amount of time between client retries
4947     CONF.setLong("hbase.client.pause", 15 * 1000);
4948 
4949     // This size should make it so we always split using the addContent
4950     // below. After adding all data, the first region is 1.3M
4951     CONF.setLong(HConstants.HREGION_MAX_FILESIZE, 1024 * 128);
4952     return CONF;
4953   }
4954 
4955   /**
4956    * @param tableName
4957    * @param callingMethod
4958    * @param conf
4959    * @param families
4960    * @throws IOException
4961    * @return A region on which you must call
4962    *         {@link HRegion#closeHRegion(HRegion)} when done.
4963    */
4964   public static HRegion initHRegion(TableName tableName, String callingMethod, Configuration conf,
4965       byte[]... families) throws IOException {
4966     return initHRegion(tableName.getName(), null, null, callingMethod, conf, false, families);
4967   }
4968 
4969   /**
4970    * @param tableName
4971    * @param callingMethod
4972    * @param conf
4973    * @param families
4974    * @throws IOException
4975    * @return A region on which you must call
4976    *         {@link HRegion#closeHRegion(HRegion)} when done.
4977    */
4978   public static HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf,
4979       byte[]... families) throws IOException {
4980     return initHRegion(tableName, null, null, callingMethod, conf, false, families);
4981   }
4982 
4983   /**
4984    * @param tableName
4985    * @param callingMethod
4986    * @param conf
4987    * @param isReadOnly
4988    * @param families
4989    * @throws IOException
4990    * @return A region on which you must call
4991    *         {@link HRegion#closeHRegion(HRegion)} when done.
4992    */
4993   public static HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf,
4994       boolean isReadOnly, byte[]... families) throws IOException {
4995     return initHRegion(tableName, null, null, callingMethod, conf, isReadOnly, families);
4996   }
4997 
4998   public static HRegion initHRegion(byte[] tableName, byte[] startKey, byte[] stopKey,
4999       String callingMethod, Configuration conf, boolean isReadOnly, byte[]... families)
5000       throws IOException {
5001     return initHRegion(tableName, startKey, stopKey, callingMethod, conf, isReadOnly,
5002         Durability.SYNC_WAL, null, families);
5003   }
5004 
5005   /**
5006    * @param tableName
5007    * @param startKey
5008    * @param stopKey
5009    * @param callingMethod
5010    * @param conf
5011    * @param isReadOnly
5012    * @param families
5013    * @throws IOException
5014    * @return A region on which you must call
5015    *         {@link HRegion#closeHRegion(HRegion)} when done.
5016    */
5017   public static HRegion initHRegion(byte[] tableName, byte[] startKey, byte[] stopKey,
5018       String callingMethod, Configuration conf, boolean isReadOnly, Durability durability,
5019       WAL wal, byte[]... families) throws IOException {
5020     return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, callingMethod, conf,
5021         isReadOnly, durability, wal, families);
5022   }
5023 
5024   /**
5025    * Assert that the passed in Cell has expected contents for the specified row,
5026    * column & timestamp.
5027    */
5028   private void checkOneCell(Cell kv, byte[] cf, int rowIdx, int colIdx, long ts) {
5029     String ctx = "rowIdx=" + rowIdx + "; colIdx=" + colIdx + "; ts=" + ts;
5030     assertEquals("Row mismatch which checking: " + ctx, "row:" + rowIdx,
5031         Bytes.toString(CellUtil.cloneRow(kv)));
5032     assertEquals("ColumnFamily mismatch while checking: " + ctx, Bytes.toString(cf),
5033         Bytes.toString(CellUtil.cloneFamily(kv)));
5034     assertEquals("Column qualifier mismatch while checking: " + ctx, "column:" + colIdx,
5035         Bytes.toString(CellUtil.cloneQualifier(kv)));
5036     assertEquals("Timestamp mismatch while checking: " + ctx, ts, kv.getTimestamp());
5037     assertEquals("Value mismatch while checking: " + ctx, "value-version-" + ts,
5038         Bytes.toString(CellUtil.cloneValue(kv)));
5039   }
5040 
5041   @Test (timeout=60000)
5042   public void testReverseScanner_FromMemStore_SingleCF_Normal()
5043       throws IOException {
5044     byte[] rowC = Bytes.toBytes("rowC");
5045     byte[] rowA = Bytes.toBytes("rowA");
5046     byte[] rowB = Bytes.toBytes("rowB");
5047     byte[] cf = Bytes.toBytes("CF");
5048     byte[][] families = { cf };
5049     byte[] col = Bytes.toBytes("C");
5050     long ts = 1;
5051     String method = this.getName();
5052     this.region = initHRegion(tableName, method, families);
5053     KeyValue kv1 = new KeyValue(rowC, cf, col, ts, KeyValue.Type.Put, null);
5054     KeyValue kv11 = new KeyValue(rowC, cf, col, ts + 1, KeyValue.Type.Put,
5055         null);
5056     KeyValue kv2 = new KeyValue(rowA, cf, col, ts, KeyValue.Type.Put, null);
5057     KeyValue kv3 = new KeyValue(rowB, cf, col, ts, KeyValue.Type.Put, null);
5058     Put put = null;
5059     put = new Put(rowC);
5060     put.add(kv1);
5061     put.add(kv11);
5062     region.put(put);
5063     put = new Put(rowA);
5064     put.add(kv2);
5065     region.put(put);
5066     put = new Put(rowB);
5067     put.add(kv3);
5068     region.put(put);
5069 
5070     Scan scan = new Scan(rowC);
5071     scan.setMaxVersions(5);
5072     scan.setReversed(true);
5073     InternalScanner scanner = region.getScanner(scan);
5074     List<Cell> currRow = new ArrayList<Cell>();
5075     boolean hasNext = scanner.next(currRow);
5076     assertEquals(2, currRow.size());
5077     assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC));
5078     assertTrue(hasNext);
5079     currRow.clear();
5080     hasNext = scanner.next(currRow);
5081     assertEquals(1, currRow.size());
5082     assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB));
5083     assertTrue(hasNext);
5084     currRow.clear();
5085     hasNext = scanner.next(currRow);
5086     assertEquals(1, currRow.size());
5087     assertTrue(Bytes.equals(currRow.get(0).getRow(), rowA));
5088     assertFalse(hasNext);
5089     scanner.close();
5090   }
5091 
5092   @Test (timeout=60000)
5093   public void testReverseScanner_FromMemStore_SingleCF_LargerKey()
5094       throws IOException {
5095     byte[] rowC = Bytes.toBytes("rowC");
5096     byte[] rowA = Bytes.toBytes("rowA");
5097     byte[] rowB = Bytes.toBytes("rowB");
5098     byte[] rowD = Bytes.toBytes("rowD");
5099     byte[] cf = Bytes.toBytes("CF");
5100     byte[][] families = { cf };
5101     byte[] col = Bytes.toBytes("C");
5102     long ts = 1;
5103     String method = this.getName();
5104     this.region = initHRegion(tableName, method, families);
5105     KeyValue kv1 = new KeyValue(rowC, cf, col, ts, KeyValue.Type.Put, null);
5106     KeyValue kv11 = new KeyValue(rowC, cf, col, ts + 1, KeyValue.Type.Put,
5107         null);
5108     KeyValue kv2 = new KeyValue(rowA, cf, col, ts, KeyValue.Type.Put, null);
5109     KeyValue kv3 = new KeyValue(rowB, cf, col, ts, KeyValue.Type.Put, null);
5110     Put put = null;
5111     put = new Put(rowC);
5112     put.add(kv1);
5113     put.add(kv11);
5114     region.put(put);
5115     put = new Put(rowA);
5116     put.add(kv2);
5117     region.put(put);
5118     put = new Put(rowB);
5119     put.add(kv3);
5120     region.put(put);
5121 
5122     Scan scan = new Scan(rowD);
5123     List<Cell> currRow = new ArrayList<Cell>();
5124     scan.setReversed(true);
5125     scan.setMaxVersions(5);
5126     InternalScanner scanner = region.getScanner(scan);
5127     boolean hasNext = scanner.next(currRow);
5128     assertEquals(2, currRow.size());
5129     assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC));
5130     assertTrue(hasNext);
5131     currRow.clear();
5132     hasNext = scanner.next(currRow);
5133     assertEquals(1, currRow.size());
5134     assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB));
5135     assertTrue(hasNext);
5136     currRow.clear();
5137     hasNext = scanner.next(currRow);
5138     assertEquals(1, currRow.size());
5139     assertTrue(Bytes.equals(currRow.get(0).getRow(), rowA));
5140     assertFalse(hasNext);
5141     scanner.close();
5142   }
5143 
5144   @Test (timeout=60000)
5145   public void testReverseScanner_FromMemStore_SingleCF_FullScan()
5146       throws IOException {
5147     byte[] rowC = Bytes.toBytes("rowC");
5148     byte[] rowA = Bytes.toBytes("rowA");
5149     byte[] rowB = Bytes.toBytes("rowB");
5150     byte[] cf = Bytes.toBytes("CF");
5151     byte[][] families = { cf };
5152     byte[] col = Bytes.toBytes("C");
5153     long ts = 1;
5154     String method = this.getName();
5155     this.region = initHRegion(tableName, method, families);
5156     KeyValue kv1 = new KeyValue(rowC, cf, col, ts, KeyValue.Type.Put, null);
5157     KeyValue kv11 = new KeyValue(rowC, cf, col, ts + 1, KeyValue.Type.Put,
5158         null);
5159     KeyValue kv2 = new KeyValue(rowA, cf, col, ts, KeyValue.Type.Put, null);
5160     KeyValue kv3 = new KeyValue(rowB, cf, col, ts, KeyValue.Type.Put, null);
5161     Put put = null;
5162     put = new Put(rowC);
5163     put.add(kv1);
5164     put.add(kv11);
5165     region.put(put);
5166     put = new Put(rowA);
5167     put.add(kv2);
5168     region.put(put);
5169     put = new Put(rowB);
5170     put.add(kv3);
5171     region.put(put);
5172     Scan scan = new Scan();
5173     List<Cell> currRow = new ArrayList<Cell>();
5174     scan.setReversed(true);
5175     InternalScanner scanner = region.getScanner(scan);
5176     boolean hasNext = scanner.next(currRow);
5177     assertEquals(1, currRow.size());
5178     assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC));
5179     assertTrue(hasNext);
5180     currRow.clear();
5181     hasNext = scanner.next(currRow);
5182     assertEquals(1, currRow.size());
5183     assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB));
5184     assertTrue(hasNext);
5185     currRow.clear();
5186     hasNext = scanner.next(currRow);
5187     assertEquals(1, currRow.size());
5188     assertTrue(Bytes.equals(currRow.get(0).getRow(), rowA));
5189     assertFalse(hasNext);
5190     scanner.close();
5191   }
5192 
5193   @Test (timeout=60000)
5194   public void testReverseScanner_moreRowsMayExistAfter() throws IOException {
5195     // case for "INCLUDE_AND_SEEK_NEXT_ROW & SEEK_NEXT_ROW" endless loop
5196     byte[] rowA = Bytes.toBytes("rowA");
5197     byte[] rowB = Bytes.toBytes("rowB");
5198     byte[] rowC = Bytes.toBytes("rowC");
5199     byte[] rowD = Bytes.toBytes("rowD");
5200     byte[] rowE = Bytes.toBytes("rowE");
5201     byte[] cf = Bytes.toBytes("CF");
5202     byte[][] families = { cf };
5203     byte[] col1 = Bytes.toBytes("col1");
5204     byte[] col2 = Bytes.toBytes("col2");
5205     long ts = 1;
5206     String method = this.getName();
5207     this.region = initHRegion(tableName, method, families);
5208     KeyValue kv1 = new KeyValue(rowA, cf, col1, ts, KeyValue.Type.Put, null);
5209     KeyValue kv2 = new KeyValue(rowB, cf, col1, ts, KeyValue.Type.Put, null);
5210     KeyValue kv3 = new KeyValue(rowC, cf, col1, ts, KeyValue.Type.Put, null);
5211     KeyValue kv4_1 = new KeyValue(rowD, cf, col1, ts, KeyValue.Type.Put, null);
5212     KeyValue kv4_2 = new KeyValue(rowD, cf, col2, ts, KeyValue.Type.Put, null);
5213     KeyValue kv5 = new KeyValue(rowE, cf, col1, ts, KeyValue.Type.Put, null);
5214     Put put = null;
5215     put = new Put(rowA);
5216     put.add(kv1);
5217     region.put(put);
5218     put = new Put(rowB);
5219     put.add(kv2);
5220     region.put(put);
5221     put = new Put(rowC);
5222     put.add(kv3);
5223     region.put(put);
5224     put = new Put(rowD);
5225     put.add(kv4_1);
5226     region.put(put);
5227     put = new Put(rowD);
5228     put.add(kv4_2);
5229     region.put(put);
5230     put = new Put(rowE);
5231     put.add(kv5);
5232     region.put(put);
5233     region.flush(true);
5234     Scan scan = new Scan(rowD, rowA);
5235     scan.addColumn(families[0], col1);
5236     scan.setReversed(true);
5237     List<Cell> currRow = new ArrayList<Cell>();
5238     InternalScanner scanner = region.getScanner(scan);
5239     boolean hasNext = scanner.next(currRow);
5240     assertEquals(1, currRow.size());
5241     assertTrue(Bytes.equals(currRow.get(0).getRow(), rowD));
5242     assertTrue(hasNext);
5243     currRow.clear();
5244     hasNext = scanner.next(currRow);
5245     assertEquals(1, currRow.size());
5246     assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC));
5247     assertTrue(hasNext);
5248     currRow.clear();
5249     hasNext = scanner.next(currRow);
5250     assertEquals(1, currRow.size());
5251     assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB));
5252     assertFalse(hasNext);
5253     scanner.close();
5254 
5255     scan = new Scan(rowD, rowA);
5256     scan.addColumn(families[0], col2);
5257     scan.setReversed(true);
5258     currRow.clear();
5259     scanner = region.getScanner(scan);
5260     hasNext = scanner.next(currRow);
5261     assertEquals(1, currRow.size());
5262     assertTrue(Bytes.equals(currRow.get(0).getRow(), rowD));
5263     scanner.close();
5264   }
5265 
5266   @Test (timeout=60000)
5267   public void testReverseScanner_smaller_blocksize() throws IOException {
5268     // case to ensure no conflict with HFile index optimization
5269     byte[] rowA = Bytes.toBytes("rowA");
5270     byte[] rowB = Bytes.toBytes("rowB");
5271     byte[] rowC = Bytes.toBytes("rowC");
5272     byte[] rowD = Bytes.toBytes("rowD");
5273     byte[] rowE = Bytes.toBytes("rowE");
5274     byte[] cf = Bytes.toBytes("CF");
5275     byte[][] families = { cf };
5276     byte[] col1 = Bytes.toBytes("col1");
5277     byte[] col2 = Bytes.toBytes("col2");
5278     long ts = 1;
5279     String method = this.getName();
5280     Configuration conf = new Configuration(CONF);
5281     conf.setInt("test.block.size", 1);
5282     this.region = initHRegion(tableName, method, conf, families);
5283     KeyValue kv1 = new KeyValue(rowA, cf, col1, ts, KeyValue.Type.Put, null);
5284     KeyValue kv2 = new KeyValue(rowB, cf, col1, ts, KeyValue.Type.Put, null);
5285     KeyValue kv3 = new KeyValue(rowC, cf, col1, ts, KeyValue.Type.Put, null);
5286     KeyValue kv4_1 = new KeyValue(rowD, cf, col1, ts, KeyValue.Type.Put, null);
5287     KeyValue kv4_2 = new KeyValue(rowD, cf, col2, ts, KeyValue.Type.Put, null);
5288     KeyValue kv5 = new KeyValue(rowE, cf, col1, ts, KeyValue.Type.Put, null);
5289     Put put = null;
5290     put = new Put(rowA);
5291     put.add(kv1);
5292     region.put(put);
5293     put = new Put(rowB);
5294     put.add(kv2);
5295     region.put(put);
5296     put = new Put(rowC);
5297     put.add(kv3);
5298     region.put(put);
5299     put = new Put(rowD);
5300     put.add(kv4_1);
5301     region.put(put);
5302     put = new Put(rowD);
5303     put.add(kv4_2);
5304     region.put(put);
5305     put = new Put(rowE);
5306     put.add(kv5);
5307     region.put(put);
5308     region.flush(true);
5309     Scan scan = new Scan(rowD, rowA);
5310     scan.addColumn(families[0], col1);
5311     scan.setReversed(true);
5312     List<Cell> currRow = new ArrayList<Cell>();
5313     InternalScanner scanner = region.getScanner(scan);
5314     boolean hasNext = scanner.next(currRow);
5315     assertEquals(1, currRow.size());
5316     assertTrue(Bytes.equals(currRow.get(0).getRow(), rowD));
5317     assertTrue(hasNext);
5318     currRow.clear();
5319     hasNext = scanner.next(currRow);
5320     assertEquals(1, currRow.size());
5321     assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC));
5322     assertTrue(hasNext);
5323     currRow.clear();
5324     hasNext = scanner.next(currRow);
5325     assertEquals(1, currRow.size());
5326     assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB));
5327     assertFalse(hasNext);
5328     scanner.close();
5329 
5330     scan = new Scan(rowD, rowA);
5331     scan.addColumn(families[0], col2);
5332     scan.setReversed(true);
5333     currRow.clear();
5334     scanner = region.getScanner(scan);
5335     hasNext = scanner.next(currRow);
5336     assertEquals(1, currRow.size());
5337     assertTrue(Bytes.equals(currRow.get(0).getRow(), rowD));
5338     scanner.close();
5339   }
5340 
5341   @Test (timeout=60000)
5342   public void testReverseScanner_FromMemStoreAndHFiles_MultiCFs1()
5343       throws IOException {
5344     byte[] row0 = Bytes.toBytes("row0"); // 1 kv
5345     byte[] row1 = Bytes.toBytes("row1"); // 2 kv
5346     byte[] row2 = Bytes.toBytes("row2"); // 4 kv
5347     byte[] row3 = Bytes.toBytes("row3"); // 2 kv
5348     byte[] row4 = Bytes.toBytes("row4"); // 5 kv
5349     byte[] row5 = Bytes.toBytes("row5"); // 2 kv
5350     byte[] cf1 = Bytes.toBytes("CF1");
5351     byte[] cf2 = Bytes.toBytes("CF2");
5352     byte[] cf3 = Bytes.toBytes("CF3");
5353     byte[][] families = { cf1, cf2, cf3 };
5354     byte[] col = Bytes.toBytes("C");
5355     long ts = 1;
5356     String method = this.getName();
5357     Configuration conf = new Configuration(CONF);
5358     // disable compactions in this test.
5359     conf.setInt("hbase.hstore.compactionThreshold", 10000);
5360     this.region = initHRegion(tableName, method, conf, families);
5361     // kv naming style: kv(row number) totalKvCountInThisRow seq no
5362     KeyValue kv0_1_1 = new KeyValue(row0, cf1, col, ts, KeyValue.Type.Put,
5363         null);
5364     KeyValue kv1_2_1 = new KeyValue(row1, cf2, col, ts, KeyValue.Type.Put,
5365         null);
5366     KeyValue kv1_2_2 = new KeyValue(row1, cf1, col, ts + 1,
5367         KeyValue.Type.Put, null);
5368     KeyValue kv2_4_1 = new KeyValue(row2, cf2, col, ts, KeyValue.Type.Put,
5369         null);
5370     KeyValue kv2_4_2 = new KeyValue(row2, cf1, col, ts, KeyValue.Type.Put,
5371         null);
5372     KeyValue kv2_4_3 = new KeyValue(row2, cf3, col, ts, KeyValue.Type.Put,
5373         null);
5374     KeyValue kv2_4_4 = new KeyValue(row2, cf1, col, ts + 4,
5375         KeyValue.Type.Put, null);
5376     KeyValue kv3_2_1 = new KeyValue(row3, cf2, col, ts, KeyValue.Type.Put,
5377         null);
5378     KeyValue kv3_2_2 = new KeyValue(row3, cf1, col, ts + 4,
5379         KeyValue.Type.Put, null);
5380     KeyValue kv4_5_1 = new KeyValue(row4, cf1, col, ts, KeyValue.Type.Put,
5381         null);
5382     KeyValue kv4_5_2 = new KeyValue(row4, cf3, col, ts, KeyValue.Type.Put,
5383         null);
5384     KeyValue kv4_5_3 = new KeyValue(row4, cf3, col, ts + 5,
5385         KeyValue.Type.Put, null);
5386     KeyValue kv4_5_4 = new KeyValue(row4, cf2, col, ts, KeyValue.Type.Put,
5387         null);
5388     KeyValue kv4_5_5 = new KeyValue(row4, cf1, col, ts + 3,
5389         KeyValue.Type.Put, null);
5390     KeyValue kv5_2_1 = new KeyValue(row5, cf2, col, ts, KeyValue.Type.Put,
5391         null);
5392     KeyValue kv5_2_2 = new KeyValue(row5, cf3, col, ts, KeyValue.Type.Put,
5393         null);
5394     // hfiles(cf1/cf2) :"row1"(1 kv) / "row2"(1 kv) / "row4"(2 kv)
5395     Put put = null;
5396     put = new Put(row1);
5397     put.add(kv1_2_1);
5398     region.put(put);
5399     put = new Put(row2);
5400     put.add(kv2_4_1);
5401     region.put(put);
5402     put = new Put(row4);
5403     put.add(kv4_5_4);
5404     put.add(kv4_5_5);
5405     region.put(put);
5406     region.flush(true);
5407     // hfiles(cf1/cf3) : "row1" (1 kvs) / "row2" (1 kv) / "row4" (2 kv)
5408     put = new Put(row4);
5409     put.add(kv4_5_1);
5410     put.add(kv4_5_3);
5411     region.put(put);
5412     put = new Put(row1);
5413     put.add(kv1_2_2);
5414     region.put(put);
5415     put = new Put(row2);
5416     put.add(kv2_4_4);
5417     region.put(put);
5418     region.flush(true);
5419     // hfiles(cf1/cf3) : "row2"(2 kv) / "row3"(1 kvs) / "row4" (1 kv)
5420     put = new Put(row4);
5421     put.add(kv4_5_2);
5422     region.put(put);
5423     put = new Put(row2);
5424     put.add(kv2_4_2);
5425     put.add(kv2_4_3);
5426     region.put(put);
5427     put = new Put(row3);
5428     put.add(kv3_2_2);
5429     region.put(put);
5430     region.flush(true);
5431     // memstore(cf1/cf2/cf3) : "row0" (1 kvs) / "row3" ( 1 kv) / "row5" (max)
5432     // ( 2 kv)
5433     put = new Put(row0);
5434     put.add(kv0_1_1);
5435     region.put(put);
5436     put = new Put(row3);
5437     put.add(kv3_2_1);
5438     region.put(put);
5439     put = new Put(row5);
5440     put.add(kv5_2_1);
5441     put.add(kv5_2_2);
5442     region.put(put);
5443     // scan range = ["row4", min), skip the max "row5"
5444     Scan scan = new Scan(row4);
5445     scan.setMaxVersions(5);
5446     scan.setBatch(3);
5447     scan.setReversed(true);
5448     InternalScanner scanner = region.getScanner(scan);
5449     List<Cell> currRow = new ArrayList<Cell>();
5450     boolean hasNext = false;
5451     // 1. scan out "row4" (5 kvs), "row5" can't be scanned out since not
5452     // included in scan range
5453     // "row4" takes 2 next() calls since batch=3
5454     hasNext = scanner.next(currRow);
5455     assertEquals(3, currRow.size());
5456     assertTrue(Bytes.equals(currRow.get(0).getRow(), row4));
5457     assertTrue(hasNext);
5458     currRow.clear();
5459     hasNext = scanner.next(currRow);
5460     assertEquals(2, currRow.size());
5461     assertTrue(Bytes.equals(currRow.get(0).getRow(), row4));
5462     assertTrue(hasNext);
5463     // 2. scan out "row3" (2 kv)
5464     currRow.clear();
5465     hasNext = scanner.next(currRow);
5466     assertEquals(2, currRow.size());
5467     assertTrue(Bytes.equals(currRow.get(0).getRow(), row3));
5468     assertTrue(hasNext);
5469     // 3. scan out "row2" (4 kvs)
5470     // "row2" takes 2 next() calls since batch=3
5471     currRow.clear();
5472     hasNext = scanner.next(currRow);
5473     assertEquals(3, currRow.size());
5474     assertTrue(Bytes.equals(currRow.get(0).getRow(), row2));
5475     assertTrue(hasNext);
5476     currRow.clear();
5477     hasNext = scanner.next(currRow);
5478     assertEquals(1, currRow.size());
5479     assertTrue(Bytes.equals(currRow.get(0).getRow(), row2));
5480     assertTrue(hasNext);
5481     // 4. scan out "row1" (2 kv)
5482     currRow.clear();
5483     hasNext = scanner.next(currRow);
5484     assertEquals(2, currRow.size());
5485     assertTrue(Bytes.equals(currRow.get(0).getRow(), row1));
5486     assertTrue(hasNext);
5487     // 5. scan out "row0" (1 kv)
5488     currRow.clear();
5489     hasNext = scanner.next(currRow);
5490     assertEquals(1, currRow.size());
5491     assertTrue(Bytes.equals(currRow.get(0).getRow(), row0));
5492     assertFalse(hasNext);
5493 
5494     scanner.close();
5495   }
5496 
5497   @Test (timeout=60000)
5498   public void testReverseScanner_FromMemStoreAndHFiles_MultiCFs2()
5499       throws IOException {
5500     byte[] row1 = Bytes.toBytes("row1");
5501     byte[] row2 = Bytes.toBytes("row2");
5502     byte[] row3 = Bytes.toBytes("row3");
5503     byte[] row4 = Bytes.toBytes("row4");
5504     byte[] cf1 = Bytes.toBytes("CF1");
5505     byte[] cf2 = Bytes.toBytes("CF2");
5506     byte[] cf3 = Bytes.toBytes("CF3");
5507     byte[] cf4 = Bytes.toBytes("CF4");
5508     byte[][] families = { cf1, cf2, cf3, cf4 };
5509     byte[] col = Bytes.toBytes("C");
5510     long ts = 1;
5511     String method = this.getName();
5512     Configuration conf = new Configuration(CONF);
5513     // disable compactions in this test.
5514     conf.setInt("hbase.hstore.compactionThreshold", 10000);
5515     this.region = initHRegion(tableName, method, conf, families);
5516     KeyValue kv1 = new KeyValue(row1, cf1, col, ts, KeyValue.Type.Put, null);
5517     KeyValue kv2 = new KeyValue(row2, cf2, col, ts, KeyValue.Type.Put, null);
5518     KeyValue kv3 = new KeyValue(row3, cf3, col, ts, KeyValue.Type.Put, null);
5519     KeyValue kv4 = new KeyValue(row4, cf4, col, ts, KeyValue.Type.Put, null);
5520     // storefile1
5521     Put put = new Put(row1);
5522     put.add(kv1);
5523     region.put(put);
5524     region.flush(true);
5525     // storefile2
5526     put = new Put(row2);
5527     put.add(kv2);
5528     region.put(put);
5529     region.flush(true);
5530     // storefile3
5531     put = new Put(row3);
5532     put.add(kv3);
5533     region.put(put);
5534     region.flush(true);
5535     // memstore
5536     put = new Put(row4);
5537     put.add(kv4);
5538     region.put(put);
5539     // scan range = ["row4", min)
5540     Scan scan = new Scan(row4);
5541     scan.setReversed(true);
5542     scan.setBatch(10);
5543     InternalScanner scanner = region.getScanner(scan);
5544     List<Cell> currRow = new ArrayList<Cell>();
5545     boolean hasNext = scanner.next(currRow);
5546     assertEquals(1, currRow.size());
5547     assertTrue(Bytes.equals(currRow.get(0).getRow(), row4));
5548     assertTrue(hasNext);
5549     currRow.clear();
5550     hasNext = scanner.next(currRow);
5551     assertEquals(1, currRow.size());
5552     assertTrue(Bytes.equals(currRow.get(0).getRow(), row3));
5553     assertTrue(hasNext);
5554     currRow.clear();
5555     hasNext = scanner.next(currRow);
5556     assertEquals(1, currRow.size());
5557     assertTrue(Bytes.equals(currRow.get(0).getRow(), row2));
5558     assertTrue(hasNext);
5559     currRow.clear();
5560     hasNext = scanner.next(currRow);
5561     assertEquals(1, currRow.size());
5562     assertTrue(Bytes.equals(currRow.get(0).getRow(), row1));
5563     assertFalse(hasNext);
5564   }
5565 
5566   /**
5567    * Test for HBASE-14497: Reverse Scan threw StackOverflow caused by readPt checking
5568    */
5569   @Test (timeout = 60000)
5570   public void testReverseScanner_StackOverflow() throws IOException {
5571     byte[] cf1 = Bytes.toBytes("CF1");
5572     byte[][] families = {cf1};
5573     byte[] col = Bytes.toBytes("C");
5574     String method = this.getName();
5575     this.region = initHRegion(tableName, method, CONF, families);
5576     // setup with one storefile and one memstore, to create scanner and get an earlier readPt
5577     Put put = new Put(Bytes.toBytes("19998"));
5578     put.add(cf1, col, Bytes.toBytes("val"));
5579     region.put(put);
5580     region.flushcache(true, true);
5581     Put put2 = new Put(Bytes.toBytes("19997"));
5582     put2.add(cf1, col, Bytes.toBytes("val"));
5583     region.put(put2);
5584 
5585     Scan scan = new Scan(Bytes.toBytes("19998"));
5586     scan.setReversed(true);
5587     InternalScanner scanner = region.getScanner(scan);
5588 
5589     // create one storefile contains many rows will be skipped
5590     // to check StoreFileScanner.seekToPreviousRow
5591     for (int i = 10000; i < 20000; i++) {
5592       Put p = new Put(Bytes.toBytes("" + i));
5593       p.add(cf1, col, Bytes.toBytes("" + i));
5594       region.put(p);
5595     }
5596     region.flushcache(true, true);
5597 
5598     // create one memstore contains many rows will be skipped
5599     // to check MemStoreScanner.seekToPreviousRow
5600     for (int i = 10000; i < 20000; i++) {
5601       Put p = new Put(Bytes.toBytes("" + i));
5602       p.add(cf1, col, Bytes.toBytes("" + i));
5603       region.put(p);
5604     }
5605 
5606     List<Cell> currRow = new ArrayList<>();
5607     boolean hasNext;
5608     do {
5609       hasNext = scanner.next(currRow);
5610     } while (hasNext);
5611     assertEquals(2, currRow.size());
5612     assertArrayEquals(Bytes.toBytes("19998"), currRow.get(0).getRow());
5613     assertArrayEquals(Bytes.toBytes("19997"), currRow.get(1).getRow());
5614   }
5615 
5616   @Test (timeout=60000)
5617   public void testSplitRegionWithReverseScan() throws IOException {
5618     byte [] tableName = Bytes.toBytes("testSplitRegionWithReverseScan");
5619     byte [] qualifier = Bytes.toBytes("qualifier");
5620     Configuration hc = initSplit();
5621     int numRows = 3;
5622     byte [][] families = {fam1};
5623 
5624     //Setting up region
5625     String method = this.getName();
5626     this.region = initHRegion(tableName, method, hc, families);
5627 
5628     //Put data in region
5629     int startRow = 100;
5630     putData(startRow, numRows, qualifier, families);
5631     int splitRow = startRow + numRows;
5632     putData(splitRow, numRows, qualifier, families);
5633     region.flush(true);
5634 
5635     HRegion [] regions = null;
5636     regions = splitRegion(region, Bytes.toBytes("" + splitRow));
5637     //Opening the regions returned.
5638     for (int i = 0; i < regions.length; i++) {
5639       regions[i] = HRegion.openHRegion(regions[i], null);
5640     }
5641     //Verifying that the region has been split
5642     assertEquals(2, regions.length);
5643 
5644     //Verifying that all data is still there and that data is in the right
5645     //place
5646     verifyData(regions[0], startRow, numRows, qualifier, families);
5647     verifyData(regions[1], splitRow, numRows, qualifier, families);
5648 
5649     //fire the reverse scan1:  top range, and larger than the last row
5650     Scan scan = new Scan(Bytes.toBytes(String.valueOf(startRow + 10 * numRows)));
5651     scan.setReversed(true);
5652     InternalScanner scanner = regions[1].getScanner(scan);
5653     List<Cell> currRow = new ArrayList<Cell>();
5654     boolean more = false;
5655     int verify = startRow + 2 * numRows - 1;
5656     do {
5657       more = scanner.next(currRow);
5658       assertEquals(Bytes.toString(currRow.get(0).getRow()), verify + "");
5659       verify--;
5660       currRow.clear();
5661     } while(more);
5662     assertEquals(verify, startRow + numRows - 1);
5663     scanner.close();
5664     //fire the reverse scan2:  top range, and equals to the last row
5665     scan = new Scan(Bytes.toBytes(String.valueOf(startRow + 2 * numRows - 1)));
5666     scan.setReversed(true);
5667     scanner = regions[1].getScanner(scan);
5668     verify = startRow + 2 * numRows - 1;
5669     do {
5670       more = scanner.next(currRow);
5671       assertEquals(Bytes.toString(currRow.get(0).getRow()), verify + "");
5672       verify--;
5673       currRow.clear();
5674     } while(more);
5675     assertEquals(verify, startRow + numRows - 1);
5676     scanner.close();
5677     //fire the reverse scan3:  bottom range, and larger than the last row
5678     scan = new Scan(Bytes.toBytes(String.valueOf(startRow + numRows)));
5679     scan.setReversed(true);
5680     scanner = regions[0].getScanner(scan);
5681     verify = startRow + numRows - 1;
5682     do {
5683       more = scanner.next(currRow);
5684       assertEquals(Bytes.toString(currRow.get(0).getRow()), verify + "");
5685       verify--;
5686       currRow.clear();
5687     } while(more);
5688     assertEquals(verify, 99);
5689     scanner.close();
5690     //fire the reverse scan4:  bottom range, and equals to the last row
5691     scan = new Scan(Bytes.toBytes(String.valueOf(startRow + numRows - 1)));
5692     scan.setReversed(true);
5693     scanner = regions[0].getScanner(scan);
5694     verify = startRow + numRows - 1;
5695     do {
5696       more = scanner.next(currRow);
5697       assertEquals(Bytes.toString(currRow.get(0).getRow()), verify + "");
5698       verify--;
5699       currRow.clear();
5700     } while(more);
5701     assertEquals(verify, startRow - 1);
5702     scanner.close();
5703   }
5704 
5705   @Test
5706   public void testWriteRequestsCounter() throws IOException {
5707     byte[] fam = Bytes.toBytes("info");
5708     byte[][] families = { fam };
5709     this.region = initHRegion(tableName, method, CONF, families);
5710 
5711     Assert.assertEquals(0L, region.getWriteRequestsCount());
5712 
5713     Put put = new Put(row);
5714     put.add(fam, fam, fam);
5715 
5716     Assert.assertEquals(0L, region.getWriteRequestsCount());
5717     region.put(put);
5718     Assert.assertEquals(1L, region.getWriteRequestsCount());
5719     region.put(put);
5720     Assert.assertEquals(2L, region.getWriteRequestsCount());
5721     region.put(put);
5722     Assert.assertEquals(3L, region.getWriteRequestsCount());
5723 
5724     region.delete(new Delete(row));
5725     Assert.assertEquals(4L, region.getWriteRequestsCount());
5726   }
5727 
5728   @Test
5729   public void testOpenRegionWrittenToWAL() throws Exception {
5730     final ServerName serverName = ServerName.valueOf("testOpenRegionWrittenToWAL", 100, 42);
5731     final RegionServerServices rss = spy(TEST_UTIL.createMockRegionServerService(serverName));
5732 
5733     HTableDescriptor htd
5734         = new HTableDescriptor(TableName.valueOf("testOpenRegionWrittenToWAL"));
5735     htd.addFamily(new HColumnDescriptor(fam1));
5736     htd.addFamily(new HColumnDescriptor(fam2));
5737 
5738     HRegionInfo hri = new HRegionInfo(htd.getTableName(),
5739       HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY);
5740 
5741     // open the region w/o rss and wal and flush some files
5742     region = HRegion.createHRegion(hri, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(),
5743         htd);
5744     assertNotNull(region);
5745 
5746     // create a file in fam1 for the region before opening in OpenRegionHandler
5747     region.put(new Put(Bytes.toBytes("a")).add(fam1, fam1, fam1));
5748     region.flush(true);
5749     HBaseTestingUtility.closeRegionAndWAL(region);
5750 
5751     ArgumentCaptor<WALEdit> editCaptor = ArgumentCaptor.forClass(WALEdit.class);
5752 
5753     // capture append() calls
5754     WAL wal = mockWAL();
5755     when(rss.getWAL((HRegionInfo) any())).thenReturn(wal);
5756 
5757     region = HRegion.openHRegion(hri, htd, rss.getWAL(hri),
5758       TEST_UTIL.getConfiguration(), rss, null);
5759 
5760     verify(wal, times(1)).append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any()
5761       , editCaptor.capture(), anyBoolean());
5762 
5763     WALEdit edit = editCaptor.getValue();
5764     assertNotNull(edit);
5765     assertNotNull(edit.getCells());
5766     assertEquals(1, edit.getCells().size());
5767     RegionEventDescriptor desc = WALEdit.getRegionEventDescriptor(edit.getCells().get(0));
5768     assertNotNull(desc);
5769 
5770     LOG.info("RegionEventDescriptor from WAL: " + desc);
5771 
5772     assertEquals(RegionEventDescriptor.EventType.REGION_OPEN, desc.getEventType());
5773     assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getName()));
5774     assertTrue(Bytes.equals(desc.getEncodedRegionName().toByteArray(),
5775       hri.getEncodedNameAsBytes()));
5776     assertTrue(desc.getLogSequenceNumber() > 0);
5777     assertEquals(serverName, ProtobufUtil.toServerName(desc.getServer()));
5778     assertEquals(2, desc.getStoresCount());
5779 
5780     StoreDescriptor store = desc.getStores(0);
5781     assertTrue(Bytes.equals(store.getFamilyName().toByteArray(), fam1));
5782     assertEquals(store.getStoreHomeDir(), Bytes.toString(fam1));
5783     assertEquals(1, store.getStoreFileCount()); // 1store file
5784     assertFalse(store.getStoreFile(0).contains("/")); // ensure path is relative
5785 
5786     store = desc.getStores(1);
5787     assertTrue(Bytes.equals(store.getFamilyName().toByteArray(), fam2));
5788     assertEquals(store.getStoreHomeDir(), Bytes.toString(fam2));
5789     assertEquals(0, store.getStoreFileCount()); // no store files
5790   }
5791 
5792   // Helper for test testOpenRegionWrittenToWALForLogReplay
5793   static class HRegionWithSeqId extends HRegion {
5794     public HRegionWithSeqId(final Path tableDir, final WAL wal, final FileSystem fs,
5795         final Configuration confParam, final HRegionInfo regionInfo,
5796         final HTableDescriptor htd, final RegionServerServices rsServices) {
5797       super(tableDir, wal, fs, confParam, regionInfo, htd, rsServices);
5798     }
5799     @Override
5800     protected long getNextSequenceId(WAL wal) throws IOException {
5801       return 42;
5802     }
5803   }
5804 
5805   @Test
5806   public void testFlushedFileWithNoTags() throws Exception {
5807     String method = "testFlushedFileWithNoTags";
5808     HTableDescriptor htd = new HTableDescriptor(tableName);
5809     htd.addFamily(new HColumnDescriptor(fam1));
5810     region = initHRegion(Bytes.toBytes(method), method, TEST_UTIL.getConfiguration(), fam1);
5811     Put put = new Put(Bytes.toBytes("a-b-0-0"));
5812     put.addColumn(fam1, qual1, Bytes.toBytes("c1-value"));
5813     region.put(put);
5814     region.flush(true);
5815     Store store = region.getStore(fam1);
5816     Collection<StoreFile> storefiles = store.getStorefiles();
5817     for (StoreFile sf : storefiles) {
5818       assertFalse("Tags should not be present "
5819           ,sf.getReader().getHFileReader().getFileContext().isIncludesTags());
5820     }
5821   }
5822 
5823   @Test
5824   public void testOpenRegionWrittenToWALForLogReplay() throws Exception {
5825     // similar to the above test but with distributed log replay
5826     final ServerName serverName = ServerName.valueOf("testOpenRegionWrittenToWALForLogReplay",
5827       100, 42);
5828     final RegionServerServices rss = spy(TEST_UTIL.createMockRegionServerService(serverName));
5829 
5830     HTableDescriptor htd
5831         = new HTableDescriptor(TableName.valueOf("testOpenRegionWrittenToWALForLogReplay"));
5832     htd.addFamily(new HColumnDescriptor(fam1));
5833     htd.addFamily(new HColumnDescriptor(fam2));
5834 
5835     HRegionInfo hri = new HRegionInfo(htd.getTableName(),
5836       HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY);
5837 
5838     // open the region w/o rss and wal and flush some files
5839     region = HRegion.createHRegion(hri, TEST_UTIL.getDataTestDir(), TEST_UTIL
5840              .getConfiguration(), htd);
5841     assertNotNull(region);
5842 
5843     // create a file in fam1 for the region before opening in OpenRegionHandler
5844     region.put(new Put(Bytes.toBytes("a")).add(fam1, fam1, fam1));
5845     region.flush(true);
5846     HRegion.closeHRegion(region);
5847 
5848     ArgumentCaptor<WALEdit> editCaptor = ArgumentCaptor.forClass(WALEdit.class);
5849 
5850     // capture append() calls
5851     WAL wal = mockWAL();
5852     when(rss.getWAL((HRegionInfo) any())).thenReturn(wal);
5853 
5854     // add the region to recovering regions
5855     HashMap<String, Region> recoveringRegions = Maps.newHashMap();
5856     recoveringRegions.put(region.getRegionInfo().getEncodedName(), null);
5857     when(rss.getRecoveringRegions()).thenReturn(recoveringRegions);
5858 
5859     Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
5860     conf.set(HConstants.REGION_IMPL, HRegionWithSeqId.class.getName());
5861     region = HRegion.openHRegion(hri, htd, rss.getWAL(hri),
5862       conf, rss, null);
5863 
5864     // verify that we have not appended region open event to WAL because this region is still
5865     // recovering
5866     verify(wal, times(0)).append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any()
5867       , editCaptor.capture(), anyBoolean());
5868 
5869     // not put the region out of recovering state
5870     new FinishRegionRecoveringHandler(rss, region.getRegionInfo().getEncodedName(), "/foo")
5871       .prepare().process();
5872 
5873     // now we should have put the entry
5874     verify(wal, times(1)).append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any()
5875       , editCaptor.capture(), anyBoolean());
5876 
5877     WALEdit edit = editCaptor.getValue();
5878     assertNotNull(edit);
5879     assertNotNull(edit.getCells());
5880     assertEquals(1, edit.getCells().size());
5881     RegionEventDescriptor desc = WALEdit.getRegionEventDescriptor(edit.getCells().get(0));
5882     assertNotNull(desc);
5883 
5884     LOG.info("RegionEventDescriptor from WAL: " + desc);
5885 
5886     assertEquals(RegionEventDescriptor.EventType.REGION_OPEN, desc.getEventType());
5887     assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getName()));
5888     assertTrue(Bytes.equals(desc.getEncodedRegionName().toByteArray(),
5889       hri.getEncodedNameAsBytes()));
5890     assertTrue(desc.getLogSequenceNumber() > 0);
5891     assertEquals(serverName, ProtobufUtil.toServerName(desc.getServer()));
5892     assertEquals(2, desc.getStoresCount());
5893 
5894     StoreDescriptor store = desc.getStores(0);
5895     assertTrue(Bytes.equals(store.getFamilyName().toByteArray(), fam1));
5896     assertEquals(store.getStoreHomeDir(), Bytes.toString(fam1));
5897     assertEquals(1, store.getStoreFileCount()); // 1store file
5898     assertFalse(store.getStoreFile(0).contains("/")); // ensure path is relative
5899 
5900     store = desc.getStores(1);
5901     assertTrue(Bytes.equals(store.getFamilyName().toByteArray(), fam2));
5902     assertEquals(store.getStoreHomeDir(), Bytes.toString(fam2));
5903     assertEquals(0, store.getStoreFileCount()); // no store files
5904   }
5905 
5906   /**
5907    * Utility method to setup a WAL mock.
5908    * Needs to do the bit where we close latch on the WALKey on append else test hangs.
5909    * @return
5910    * @throws IOException
5911    */
5912   private WAL mockWAL() throws IOException {
5913     WAL wal = mock(WAL.class);
5914     Mockito.when(wal.append((HTableDescriptor)Mockito.any(), (HRegionInfo)Mockito.any(),
5915         (WALKey)Mockito.any(), (WALEdit)Mockito.any(), Mockito.anyBoolean())).
5916       thenAnswer(new Answer<Long>() {
5917         @Override
5918         public Long answer(InvocationOnMock invocation) throws Throwable {
5919           WALKey key = invocation.getArgumentAt(2, WALKey.class);
5920           MultiVersionConcurrencyControl.WriteEntry we = key.getMvcc().begin();
5921           key.setWriteEntry(we);
5922           return 1L;
5923         }
5924 
5925     });
5926     return wal;
5927   }
5928 
5929   @Test
5930   public void testCloseRegionWrittenToWAL() throws Exception {
5931 
5932     Path rootDir = new Path(dir + name.getMethodName());
5933     FSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootDir);
5934 
5935     final ServerName serverName = ServerName.valueOf("testCloseRegionWrittenToWAL", 100, 42);
5936     final RegionServerServices rss = spy(TEST_UTIL.createMockRegionServerService(serverName));
5937 
5938     HTableDescriptor htd
5939     = new HTableDescriptor(TableName.valueOf("testOpenRegionWrittenToWAL"));
5940     htd.addFamily(new HColumnDescriptor(fam1));
5941     htd.addFamily(new HColumnDescriptor(fam2));
5942 
5943     final HRegionInfo hri = new HRegionInfo(htd.getTableName(),
5944       HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY);
5945 
5946     ArgumentCaptor<WALEdit> editCaptor = ArgumentCaptor.forClass(WALEdit.class);
5947 
5948     // capture append() calls
5949     WAL wal = mockWAL();
5950     when(rss.getWAL((HRegionInfo) any())).thenReturn(wal);
5951 
5952 
5953     // create and then open a region first so that it can be closed later
5954     region = HRegion.createHRegion(hri, rootDir, TEST_UTIL.getConfiguration(), htd, rss.getWAL(hri));
5955     region = HRegion.openHRegion(hri, htd, rss.getWAL(hri),
5956       TEST_UTIL.getConfiguration(), rss, null);
5957 
5958     // close the region
5959     region.close(false);
5960 
5961     // 2 times, one for region open, the other close region
5962     verify(wal, times(2)).append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any(),
5963       editCaptor.capture(), anyBoolean());
5964 
5965     WALEdit edit = editCaptor.getAllValues().get(1);
5966     assertNotNull(edit);
5967     assertNotNull(edit.getCells());
5968     assertEquals(1, edit.getCells().size());
5969     RegionEventDescriptor desc = WALEdit.getRegionEventDescriptor(edit.getCells().get(0));
5970     assertNotNull(desc);
5971 
5972     LOG.info("RegionEventDescriptor from WAL: " + desc);
5973 
5974     assertEquals(RegionEventDescriptor.EventType.REGION_CLOSE, desc.getEventType());
5975     assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getName()));
5976     assertTrue(Bytes.equals(desc.getEncodedRegionName().toByteArray(),
5977       hri.getEncodedNameAsBytes()));
5978     assertTrue(desc.getLogSequenceNumber() > 0);
5979     assertEquals(serverName, ProtobufUtil.toServerName(desc.getServer()));
5980     assertEquals(2, desc.getStoresCount());
5981 
5982     StoreDescriptor store = desc.getStores(0);
5983     assertTrue(Bytes.equals(store.getFamilyName().toByteArray(), fam1));
5984     assertEquals(store.getStoreHomeDir(), Bytes.toString(fam1));
5985     assertEquals(0, store.getStoreFileCount()); // no store files
5986 
5987     store = desc.getStores(1);
5988     assertTrue(Bytes.equals(store.getFamilyName().toByteArray(), fam2));
5989     assertEquals(store.getStoreHomeDir(), Bytes.toString(fam2));
5990     assertEquals(0, store.getStoreFileCount()); // no store files
5991   }
5992 
5993   /**
5994    * Test RegionTooBusyException thrown when region is busy
5995    */
5996   @Test (timeout=24000)
5997   public void testRegionTooBusy() throws IOException {
5998     String method = "testRegionTooBusy";
5999     byte[] tableName = Bytes.toBytes(method);
6000     byte[] family = Bytes.toBytes("family");
6001     long defaultBusyWaitDuration = CONF.getLong("hbase.busy.wait.duration",
6002       HRegion.DEFAULT_BUSY_WAIT_DURATION);
6003     CONF.setLong("hbase.busy.wait.duration", 1000);
6004     region = initHRegion(tableName, method, CONF, family);
6005     final AtomicBoolean stopped = new AtomicBoolean(true);
6006     Thread t = new Thread(new Runnable() {
6007       @Override
6008       public void run() {
6009         try {
6010           region.lock.writeLock().lock();
6011           stopped.set(false);
6012           while (!stopped.get()) {
6013             Thread.sleep(100);
6014           }
6015         } catch (InterruptedException ie) {
6016         } finally {
6017           region.lock.writeLock().unlock();
6018         }
6019       }
6020     });
6021     t.start();
6022     Get get = new Get(row);
6023     try {
6024       while (stopped.get()) {
6025         Thread.sleep(100);
6026       }
6027       region.get(get);
6028       fail("Should throw RegionTooBusyException");
6029     } catch (InterruptedException ie) {
6030       fail("test interrupted");
6031     } catch (RegionTooBusyException e) {
6032       // Good, expected
6033     } finally {
6034       stopped.set(true);
6035       try {
6036         t.join();
6037       } catch (Throwable e) {
6038       }
6039 
6040       HRegion.closeHRegion(region);
6041       region = null;
6042       CONF.setLong("hbase.busy.wait.duration", defaultBusyWaitDuration);
6043     }
6044   }
6045 
6046   @Test
6047   public void testCellTTLs() throws IOException {
6048     IncrementingEnvironmentEdge edge = new IncrementingEnvironmentEdge();
6049     EnvironmentEdgeManager.injectEdge(edge);
6050 
6051     final byte[] row = Bytes.toBytes("testRow");
6052     final byte[] q1 = Bytes.toBytes("q1");
6053     final byte[] q2 = Bytes.toBytes("q2");
6054     final byte[] q3 = Bytes.toBytes("q3");
6055     final byte[] q4 = Bytes.toBytes("q4");
6056 
6057     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testCellTTLs"));
6058     HColumnDescriptor hcd = new HColumnDescriptor(fam1);
6059     hcd.setTimeToLive(10); // 10 seconds
6060     htd.addFamily(hcd);
6061 
6062     Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
6063     conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MIN_FORMAT_VERSION_WITH_TAGS);
6064 
6065     region = HRegion.createHRegion(new HRegionInfo(htd.getTableName(),
6066         HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY),
6067       TEST_UTIL.getDataTestDir(), conf, htd);
6068     assertNotNull(region);
6069     long now = EnvironmentEdgeManager.currentTime();
6070     // Add a cell that will expire in 5 seconds via cell TTL
6071     region.put(new Put(row).add(new KeyValue(row, fam1, q1, now,
6072       HConstants.EMPTY_BYTE_ARRAY, new Tag[] {
6073         // TTL tags specify ts in milliseconds
6074         new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(5000L)) } )));
6075     // Add a cell that will expire after 10 seconds via family setting
6076     region.put(new Put(row).add(fam1, q2, now, HConstants.EMPTY_BYTE_ARRAY));
6077     // Add a cell that will expire in 15 seconds via cell TTL
6078     region.put(new Put(row).add(new KeyValue(row, fam1, q3, now + 10000 - 1,
6079       HConstants.EMPTY_BYTE_ARRAY, new Tag[] {
6080         // TTL tags specify ts in milliseconds
6081         new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(5000L)) } )));
6082     // Add a cell that will expire in 20 seconds via family setting
6083     region.put(new Put(row).add(fam1, q4, now + 10000 - 1, HConstants.EMPTY_BYTE_ARRAY));
6084 
6085     // Flush so we are sure store scanning gets this right
6086     region.flush(true);
6087 
6088     // A query at time T+0 should return all cells
6089     Result r = region.get(new Get(row));
6090     assertNotNull(r.getValue(fam1, q1));
6091     assertNotNull(r.getValue(fam1, q2));
6092     assertNotNull(r.getValue(fam1, q3));
6093     assertNotNull(r.getValue(fam1, q4));
6094 
6095     // Increment time to T+5 seconds
6096     edge.incrementTime(5000);
6097 
6098     r = region.get(new Get(row));
6099     assertNull(r.getValue(fam1, q1));
6100     assertNotNull(r.getValue(fam1, q2));
6101     assertNotNull(r.getValue(fam1, q3));
6102     assertNotNull(r.getValue(fam1, q4));
6103 
6104     // Increment time to T+10 seconds
6105     edge.incrementTime(5000);
6106 
6107     r = region.get(new Get(row));
6108     assertNull(r.getValue(fam1, q1));
6109     assertNull(r.getValue(fam1, q2));
6110     assertNotNull(r.getValue(fam1, q3));
6111     assertNotNull(r.getValue(fam1, q4));
6112 
6113     // Increment time to T+15 seconds
6114     edge.incrementTime(5000);
6115 
6116     r = region.get(new Get(row));
6117     assertNull(r.getValue(fam1, q1));
6118     assertNull(r.getValue(fam1, q2));
6119     assertNull(r.getValue(fam1, q3));
6120     assertNotNull(r.getValue(fam1, q4));
6121 
6122     // Increment time to T+20 seconds
6123     edge.incrementTime(10000);
6124 
6125     r = region.get(new Get(row));
6126     assertNull(r.getValue(fam1, q1));
6127     assertNull(r.getValue(fam1, q2));
6128     assertNull(r.getValue(fam1, q3));
6129     assertNull(r.getValue(fam1, q4));
6130 
6131     // Fun with disappearing increments
6132 
6133     // Start at 1
6134     region.put(new Put(row).add(fam1, q1, Bytes.toBytes(1L)));
6135     r = region.get(new Get(row));
6136     byte[] val = r.getValue(fam1, q1);
6137     assertNotNull(val);
6138     assertEquals(Bytes.toLong(val), 1L);
6139 
6140     // Increment with a TTL of 5 seconds
6141     Increment incr = new Increment(row).addColumn(fam1, q1, 1L);
6142     incr.setTTL(5000);
6143     region.increment(incr); // 2
6144 
6145     // New value should be 2
6146     r = region.get(new Get(row));
6147     val = r.getValue(fam1, q1);
6148     assertNotNull(val);
6149     assertEquals(Bytes.toLong(val), 2L);
6150 
6151     // Increment time to T+25 seconds
6152     edge.incrementTime(5000);
6153 
6154     // Value should be back to 1
6155     r = region.get(new Get(row));
6156     val = r.getValue(fam1, q1);
6157     assertNotNull(val);
6158     assertEquals(Bytes.toLong(val), 1L);
6159 
6160     // Increment time to T+30 seconds
6161     edge.incrementTime(5000);
6162 
6163     // Original value written at T+20 should be gone now via family TTL
6164     r = region.get(new Get(row));
6165     assertNull(r.getValue(fam1, q1));
6166   }
6167 
6168   @Test
6169   public void testIncrementTimestampsAreMonotonic() throws IOException {
6170     region = initHRegion(tableName, name.getMethodName(), CONF, fam1);
6171     ManualEnvironmentEdge edge = new ManualEnvironmentEdge();
6172     EnvironmentEdgeManager.injectEdge(edge);
6173 
6174     edge.setValue(10);
6175     Increment inc = new Increment(row);
6176     inc.setDurability(Durability.SKIP_WAL);
6177     inc.addColumn(fam1, qual1, 1L);
6178     region.increment(inc);
6179 
6180     Result result = region.get(new Get(row));
6181     Cell c = result.getColumnLatestCell(fam1, qual1);
6182     assertNotNull(c);
6183     assertEquals(c.getTimestamp(), 10L);
6184 
6185     edge.setValue(1); // clock goes back
6186     region.increment(inc);
6187     result = region.get(new Get(row));
6188     c = result.getColumnLatestCell(fam1, qual1);
6189     assertEquals(c.getTimestamp(), 11L);
6190     assertEquals(Bytes.toLong(c.getValueArray(), c.getValueOffset(), c.getValueLength()), 2L);
6191   }
6192 
6193   @Test
6194   public void testAppendTimestampsAreMonotonic() throws IOException {
6195     region = initHRegion(tableName, name.getMethodName(), CONF, fam1);
6196     ManualEnvironmentEdge edge = new ManualEnvironmentEdge();
6197     EnvironmentEdgeManager.injectEdge(edge);
6198 
6199     edge.setValue(10);
6200     Append a = new Append(row);
6201     a.setDurability(Durability.SKIP_WAL);
6202     a.add(fam1, qual1, qual1);
6203     region.append(a);
6204 
6205     Result result = region.get(new Get(row));
6206     Cell c = result.getColumnLatestCell(fam1, qual1);
6207     assertNotNull(c);
6208     assertEquals(c.getTimestamp(), 10L);
6209 
6210     edge.setValue(1); // clock goes back
6211     region.append(a);
6212     result = region.get(new Get(row));
6213     c = result.getColumnLatestCell(fam1, qual1);
6214     assertEquals(c.getTimestamp(), 11L);
6215 
6216     byte[] expected = new byte[qual1.length*2];
6217     System.arraycopy(qual1, 0, expected, 0, qual1.length);
6218     System.arraycopy(qual1, 0, expected, qual1.length, qual1.length);
6219 
6220     assertTrue(Bytes.equals(c.getValueArray(), c.getValueOffset(), c.getValueLength(),
6221       expected, 0, expected.length));
6222   }
6223 
6224   @Test
6225   public void testCheckAndMutateTimestampsAreMonotonic() throws IOException {
6226     region = initHRegion(tableName, name.getMethodName(), CONF, fam1);
6227     ManualEnvironmentEdge edge = new ManualEnvironmentEdge();
6228     EnvironmentEdgeManager.injectEdge(edge);
6229 
6230     edge.setValue(10);
6231     Put p = new Put(row);
6232     p.setDurability(Durability.SKIP_WAL);
6233     p.addColumn(fam1, qual1, qual1);
6234     region.put(p);
6235 
6236     Result result = region.get(new Get(row));
6237     Cell c = result.getColumnLatestCell(fam1, qual1);
6238     assertNotNull(c);
6239     assertEquals(c.getTimestamp(), 10L);
6240 
6241     edge.setValue(1); // clock goes back
6242     p = new Put(row);
6243     p.setDurability(Durability.SKIP_WAL);
6244     p.addColumn(fam1, qual1, qual2);
6245     region.checkAndMutate(row, fam1, qual1, CompareOp.EQUAL, new BinaryComparator(qual1), p, false);
6246     result = region.get(new Get(row));
6247     c = result.getColumnLatestCell(fam1, qual1);
6248     assertEquals(c.getTimestamp(), 10L);
6249 
6250     assertTrue(Bytes.equals(c.getValueArray(), c.getValueOffset(), c.getValueLength(),
6251       qual2, 0, qual2.length));
6252   }
6253 
6254   @Test(timeout = 60000)
6255   public void testBatchMutateWithWrongRegionException() throws Exception {
6256     final byte[] a = Bytes.toBytes("a");
6257     final byte[] b = Bytes.toBytes("b");
6258     final byte[] c = Bytes.toBytes("c"); // exclusive
6259 
6260     int prevLockTimeout = CONF.getInt("hbase.rowlock.wait.duration", 30000);
6261     CONF.setInt("hbase.rowlock.wait.duration", 1000);
6262     region = initHRegion(tableName, a, c, name.getMethodName(), CONF, false, fam1);
6263 
6264     Mutation[] mutations = new Mutation[] {
6265         new Put(a).addImmutable(fam1, null, null),
6266         new Put(c).addImmutable(fam1, null, null), // this is outside the region boundary
6267         new Put(b).addImmutable(fam1, null, null),
6268     };
6269 
6270     OperationStatus[] status = region.batchMutate(mutations);
6271     assertEquals(status[0].getOperationStatusCode(), OperationStatusCode.SUCCESS);
6272     assertEquals(status[1].getOperationStatusCode(), OperationStatusCode.SANITY_CHECK_FAILURE);
6273     assertEquals(status[2].getOperationStatusCode(), OperationStatusCode.SUCCESS);
6274 
6275 
6276     // test with a row lock held for a long time
6277     final CountDownLatch obtainedRowLock = new CountDownLatch(1);
6278     ExecutorService exec = Executors.newFixedThreadPool(2);
6279     Future<Void> f1 = exec.submit(new Callable<Void>() {
6280       @Override
6281       public Void call() throws Exception {
6282         LOG.info("Acquiring row lock");
6283         RowLock rl = region.getRowLock(b);
6284         obtainedRowLock.countDown();
6285         LOG.info("Waiting for 5 seconds before releasing lock");
6286         Threads.sleep(5000);
6287         LOG.info("Releasing row lock");
6288         rl.release();
6289         return null;
6290       }
6291     });
6292     obtainedRowLock.await(30, TimeUnit.SECONDS);
6293 
6294     Future<Void> f2 = exec.submit(new Callable<Void>() {
6295       @Override
6296       public Void call() throws Exception {
6297         Mutation[] mutations = new Mutation[] {
6298             new Put(a).addImmutable(fam1, null, null),
6299             new Put(b).addImmutable(fam1, null, null),
6300         };
6301 
6302         // this will wait for the row lock, and it will eventually succeed
6303         OperationStatus[] status = region.batchMutate(mutations);
6304         assertEquals(status[0].getOperationStatusCode(), OperationStatusCode.SUCCESS);
6305         assertEquals(status[1].getOperationStatusCode(), OperationStatusCode.SUCCESS);
6306         return null;
6307       }
6308     });
6309 
6310     f1.get();
6311     f2.get();
6312 
6313     CONF.setInt("hbase.rowlock.wait.duration", prevLockTimeout);
6314   }
6315 
6316   @Test
6317   public void testBatchMutateWithZeroRowLockWait() throws Exception {
6318     final byte[] a = Bytes.toBytes("a");
6319     final byte[] b = Bytes.toBytes("b");
6320     final byte[] c = Bytes.toBytes("c"); // exclusive
6321 
6322     Configuration conf = new Configuration(CONF);
6323     conf.setInt("hbase.rowlock.wait.duration", 0);
6324     final HRegionInfo hri = new HRegionInfo(TableName.valueOf(tableName), a, c);
6325     final HTableDescriptor htd = new HTableDescriptor(tableName);
6326     htd.addFamily(new HColumnDescriptor(fam1));
6327     region = HRegion.createHRegion(hri, TEST_UTIL.getDataTestDir(), conf, htd, TEST_UTIL.createWal(conf, TEST_UTIL.getDataTestDir(), TEST_UTIL.getDataTestDirOnTestFS(method + ".log"), hri));
6328 
6329     Mutation[] mutations = new Mutation[] {
6330         new Put(a).addImmutable(fam1, null, null),
6331         new Put(b).addImmutable(fam1, null, null),
6332     };
6333 
6334     OperationStatus[] status = region.batchMutate(mutations);
6335     assertEquals(OperationStatusCode.SUCCESS, status[0].getOperationStatusCode());
6336     assertEquals(OperationStatusCode.SUCCESS, status[1].getOperationStatusCode());
6337 
6338 
6339     // test with a row lock held for a long time
6340     final CountDownLatch obtainedRowLock = new CountDownLatch(1);
6341     ExecutorService exec = Executors.newFixedThreadPool(2);
6342     Future<Void> f1 = exec.submit(new Callable<Void>() {
6343       @Override
6344       public Void call() throws Exception {
6345         LOG.info("Acquiring row lock");
6346         RowLock rl = region.getRowLock(b);
6347         obtainedRowLock.countDown();
6348         LOG.info("Waiting for 5 seconds before releasing lock");
6349         Threads.sleep(5000);
6350         LOG.info("Releasing row lock");
6351         rl.release();
6352         return null;
6353       }
6354     });
6355     obtainedRowLock.await(30, TimeUnit.SECONDS);
6356 
6357     Future<Void> f2 = exec.submit(new Callable<Void>() {
6358       @Override
6359       public Void call() throws Exception {
6360         Mutation[] mutations = new Mutation[] {
6361             new Put(a).addImmutable(fam1, null, null),
6362             new Put(b).addImmutable(fam1, null, null),
6363         };
6364         // when handling row b we are going to spin on the failure to get the row lock
6365         // until the lock above is released, but we will still succeed so long as that
6366         // takes less time then the test time out.
6367         OperationStatus[] status = region.batchMutate(mutations);
6368         assertEquals(OperationStatusCode.SUCCESS, status[0].getOperationStatusCode());
6369         assertEquals(OperationStatusCode.SUCCESS, status[1].getOperationStatusCode());
6370         return null;
6371       }
6372     });
6373 
6374     f1.get();
6375     f2.get();
6376   }
6377 
6378   @Test
6379   public void testCheckAndRowMutateTimestampsAreMonotonic() throws IOException {
6380     region = initHRegion(tableName, name.getMethodName(), CONF, fam1);
6381     ManualEnvironmentEdge edge = new ManualEnvironmentEdge();
6382     EnvironmentEdgeManager.injectEdge(edge);
6383 
6384     edge.setValue(10);
6385     Put p = new Put(row);
6386     p.setDurability(Durability.SKIP_WAL);
6387     p.addColumn(fam1, qual1, qual1);
6388     region.put(p);
6389 
6390     Result result = region.get(new Get(row));
6391     Cell c = result.getColumnLatestCell(fam1, qual1);
6392     assertNotNull(c);
6393     assertEquals(c.getTimestamp(), 10L);
6394 
6395     edge.setValue(1); // clock goes back
6396     p = new Put(row);
6397     p.setDurability(Durability.SKIP_WAL);
6398     p.addColumn(fam1, qual1, qual2);
6399     RowMutations rm = new RowMutations(row);
6400     rm.add(p);
6401     region.checkAndRowMutate(row, fam1, qual1, CompareOp.EQUAL, new BinaryComparator(qual1),
6402       rm, false);
6403     result = region.get(new Get(row));
6404     c = result.getColumnLatestCell(fam1, qual1);
6405     assertEquals(c.getTimestamp(), 10L);
6406 
6407     assertTrue(Bytes.equals(c.getValueArray(), c.getValueOffset(), c.getValueLength(),
6408       qual2, 0, qual2.length));
6409   }
6410 
6411   static HRegion initHRegion(byte[] tableName, String callingMethod,
6412       byte[]... families) throws IOException {
6413     return initHRegion(tableName, callingMethod, HBaseConfiguration.create(),
6414         families);
6415   }
6416 
6417   /**
6418    * HBASE-16429 Make sure no stuck if roll writer when ring buffer is filled with appends
6419    * @throws IOException if IO error occurred during test
6420    */
6421   @Test
6422   public void testWritesWhileRollWriter() throws IOException {
6423     int testCount = 10;
6424     int numRows = 1024;
6425     int numFamilies = 2;
6426     int numQualifiers = 2;
6427     final byte[][] families = new byte[numFamilies][];
6428     for (int i = 0; i < numFamilies; i++) {
6429       families[i] = Bytes.toBytes("family" + i);
6430     }
6431     final byte[][] qualifiers = new byte[numQualifiers][];
6432     for (int i = 0; i < numQualifiers; i++) {
6433       qualifiers[i] = Bytes.toBytes("qual" + i);
6434     }
6435 
6436     String method = "testWritesWhileRollWriter";
6437     CONF.setInt("hbase.regionserver.wal.disruptor.event.count", 2);
6438     this.region = initHRegion(tableName, method, CONF, families);
6439     try {
6440       List<Thread> threads = new ArrayList<Thread>();
6441       for (int i = 0; i < numRows; i++) {
6442         final int count = i;
6443         Thread t = new Thread(new Runnable() {
6444 
6445           @Override
6446           public void run() {
6447             byte[] row = Bytes.toBytes("row" + count);
6448             Put put = new Put(row);
6449             put.setDurability(Durability.SYNC_WAL);
6450             byte[] value = Bytes.toBytes(String.valueOf(count));
6451             for (byte[] family : families) {
6452               for (byte[] qualifier : qualifiers) {
6453                 put.addColumn(family, qualifier, (long) count, value);
6454               }
6455             }
6456             try {
6457               region.put(put);
6458             } catch (IOException e) {
6459               throw new RuntimeException(e);
6460             }
6461           }
6462         });
6463         threads.add(t);
6464       }
6465       for (Thread t : threads) {
6466         t.start();
6467       }
6468 
6469       for (int i = 0; i < testCount; i++) {
6470         region.getWAL().rollWriter();
6471         Thread.yield();
6472       }
6473     } finally {
6474       try {
6475         HBaseTestingUtility.closeRegionAndWAL(this.region);
6476         CONF.setInt("hbase.regionserver.wal.disruptor.event.count", 16 * 1024);
6477       } catch (DroppedSnapshotException dse) {
6478         // We could get this on way out because we interrupt the background flusher and it could
6479         // fail anywhere causing a DSE over in the background flusher... only it is not properly
6480         // dealt with so could still be memory hanging out when we get to here -- memory we can't
6481         // flush because the accounting is 'off' since original DSE.
6482       }
6483       this.region = null;
6484     }
6485   }
6486 
6487   @Test
6488   public void testMutateRow_WriteRequestCount() throws Exception {
6489     byte[] row1 = Bytes.toBytes("row1");
6490     byte[] fam1 = Bytes.toBytes("fam1");
6491     byte[] qf1 = Bytes.toBytes("qualifier");
6492     byte[] val1 = Bytes.toBytes("value1");
6493 
6494     RowMutations rm = new RowMutations(row1);
6495     Put put = new Put(row1);
6496     put.addColumn(fam1, qf1, val1);
6497     rm.add(put);
6498 
6499     this.region = initHRegion(tableName, method, CONF, fam1);
6500     long wrcBeforeMutate = this.region.writeRequestsCount.get();
6501     this.region.mutateRow(rm);
6502     long wrcAfterMutate = this.region.writeRequestsCount.get();
6503     Assert.assertEquals(wrcBeforeMutate + rm.getMutations().size(), wrcAfterMutate);
6504   }
6505 
6506   @Test
6507   public void testReverseScanShouldNotScanMemstoreIfReadPtLesser() throws Exception {
6508     byte[] cf1 = Bytes.toBytes("CF1");
6509     byte[][] families = { cf1 };
6510     byte[] col = Bytes.toBytes("C");
6511     String method = this.getName();
6512     HBaseConfiguration conf = new HBaseConfiguration();
6513     this.region = initHRegion(tableName, method, conf, families);
6514     // setup with one storefile and one memstore, to create scanner and get an earlier readPt
6515     Put put = new Put(Bytes.toBytes("19996"));
6516     put.addColumn(cf1, col, Bytes.toBytes("val"));
6517     region.put(put);
6518     Put put2 = new Put(Bytes.toBytes("19995"));
6519     put2.addColumn(cf1, col, Bytes.toBytes("val"));
6520     region.put(put2);
6521 
6522     // create a reverse scan
6523     Scan scan = new Scan(Bytes.toBytes("19996"));
6524     scan.setReversed(true);
6525     RegionScanner scanner = region.getScanner(scan);
6526 
6527     // flush the cache. This will reset the store scanner
6528     region.flushcache(true, true);
6529 
6530     // create one memstore contains many rows will be skipped
6531     // to check MemStoreScanner.seekToPreviousRow
6532     for (int i = 10000; i < 20000; i++) {
6533       Put p = new Put(Bytes.toBytes("" + i));
6534       p.addColumn(cf1, col, Bytes.toBytes("" + i));
6535       region.put(p);
6536     }
6537 
6538     List<Cell> currRow = new ArrayList<>();
6539     boolean hasNext;
6540     boolean assertDone = false;
6541     do {
6542       hasNext = scanner.next(currRow);
6543       // With HBASE-15871, after the scanner is reset the memstore scanner should not be
6544       // added here
6545       if (!assertDone) {
6546         StoreScanner current =
6547           (StoreScanner) (((RegionScannerImpl) scanner).storeHeap).getCurrentForTesting();
6548         List<KeyValueScanner> scanners = current.getAllScannersForTesting();
6549         assertEquals("There should be only one scanner the store file scanner", 1,
6550           scanners.size());
6551         assertDone = true;
6552       }
6553     } while (hasNext);
6554 
6555     assertEquals(2, currRow.size());
6556     assertEquals("19996", Bytes.toString(currRow.get(0).getRowArray(),
6557       currRow.get(0).getRowOffset(), currRow.get(0).getRowLength()));
6558     assertEquals("19995", Bytes.toString(currRow.get(1).getRowArray(),
6559       currRow.get(1).getRowOffset(), currRow.get(1).getRowLength()));
6560   }
6561 
6562   @Test
6563   public void testReverseScanWhenPutCellsAfterOpenReverseScan() throws Exception {
6564     byte[] cf1 = Bytes.toBytes("CF1");
6565     byte[][] families = { cf1 };
6566     byte[] col = Bytes.toBytes("C");
6567 
6568     HBaseConfiguration conf = new HBaseConfiguration();
6569     this.region = initHRegion(tableName, method, conf, families);
6570 
6571     Put put = new Put(Bytes.toBytes("199996"));
6572     put.addColumn(cf1, col, Bytes.toBytes("val"));
6573     region.put(put);
6574     Put put2 = new Put(Bytes.toBytes("199995"));
6575     put2.addColumn(cf1, col, Bytes.toBytes("val"));
6576     region.put(put2);
6577 
6578     // Create a reverse scan
6579     Scan scan = new Scan(Bytes.toBytes("199996"));
6580     scan.setReversed(true);
6581     RegionScanner scanner = region.getScanner(scan);
6582 
6583     // Put a lot of cells that have sequenceIDs grater than the readPt of the reverse scan
6584     for (int i = 100000; i < 200000; i++) {
6585       Put p = new Put(Bytes.toBytes("" + i));
6586       p.addColumn(cf1, col, Bytes.toBytes("" + i));
6587       region.put(p);
6588     }
6589     List<Cell> currRow = new ArrayList<>();
6590     boolean hasNext;
6591     do {
6592       hasNext = scanner.next(currRow);
6593     } while (hasNext);
6594 
6595     assertEquals(2, currRow.size());
6596     assertEquals("199996", Bytes.toString(currRow.get(0).getRowArray(),
6597       currRow.get(0).getRowOffset(), currRow.get(0).getRowLength()));
6598     assertEquals("199995", Bytes.toString(currRow.get(1).getRowArray(),
6599       currRow.get(1).getRowOffset(), currRow.get(1).getRowLength()));
6600   }
6601 
6602   @Test
6603   public void testCloseNoInterrupt() throws Exception {
6604     byte[] cf1 = Bytes.toBytes("CF1");
6605     byte[][] families = { cf1 };
6606     final int SLEEP_TIME = 10 * 1000;
6607 
6608     Configuration conf = new Configuration(CONF);
6609     // Disable close thread interrupt and server abort behavior
6610     conf.setBoolean(HRegion.CLOSE_WAIT_ABORT, false);
6611     conf.setInt(HRegion.CLOSE_WAIT_INTERVAL, 1000);
6612     region = initHRegion(tableName, method, conf, families);
6613 
6614     final CountDownLatch latch = new CountDownLatch(1);
6615     final AtomicBoolean holderInterrupted = new AtomicBoolean();
6616     Thread holder = new Thread(new Runnable() {
6617       @Override
6618       public void run() {
6619         try {
6620           LOG.info("Starting region operation holder");
6621           region.startRegionOperation(Operation.SCAN);
6622           latch.countDown();
6623           try {
6624             Thread.sleep(SLEEP_TIME);
6625           } catch (InterruptedException e) {
6626             LOG.info("Interrupted");
6627             holderInterrupted.set(true);
6628           }
6629         } catch (Exception e) {
6630           throw new RuntimeException(e);
6631         } finally {
6632           try {
6633             region.closeRegionOperation();
6634           } catch (IOException e) {
6635           }
6636           LOG.info("Stopped region operation holder");
6637         }
6638       }
6639     });
6640 
6641     holder.start();
6642     latch.await();
6643     region.close();
6644     region = null;
6645     holder.join();
6646 
6647     assertFalse("Region lock holder should not have been interrupted", holderInterrupted.get());
6648   }
6649 
6650   @Test
6651   public void testCloseInterrupt() throws Exception {
6652     byte[] cf1 = Bytes.toBytes("CF1");
6653     byte[][] families = { cf1 };
6654     final int SLEEP_TIME = 10 * 1000;
6655 
6656     Configuration conf = new Configuration(CONF);
6657     // Enable close thread interrupt and server abort behavior
6658     conf.setBoolean(HRegion.CLOSE_WAIT_ABORT, true);
6659     // Speed up the unit test, no need to wait default 10 seconds.
6660     conf.setInt(HRegion.CLOSE_WAIT_INTERVAL, 1000);
6661     region = initHRegion(tableName, method, conf, families);
6662 
6663     final CountDownLatch latch = new CountDownLatch(1);
6664     final AtomicBoolean holderInterrupted = new AtomicBoolean();
6665     Thread holder = new Thread(new Runnable() {
6666       @Override
6667       public void run() {
6668         try {
6669           LOG.info("Starting region operation holder");
6670           region.startRegionOperation(Operation.SCAN);
6671           latch.countDown();
6672           try {
6673             Thread.sleep(SLEEP_TIME);
6674           } catch (InterruptedException e) {
6675             LOG.info("Interrupted");
6676             holderInterrupted.set(true);
6677           }
6678         } catch (Exception e) {
6679           throw new RuntimeException(e);
6680         } finally {
6681           try {
6682             region.closeRegionOperation();
6683           } catch (IOException e) {
6684           }
6685           LOG.info("Stopped region operation holder");
6686         }
6687       }
6688     });
6689 
6690     holder.start();
6691     latch.await();
6692     region.close();
6693     region = null;
6694     holder.join();
6695 
6696     assertTrue("Region lock holder was not interrupted", holderInterrupted.get());
6697   }
6698 
6699   @Test
6700   public void testCloseAbort() throws Exception {
6701     byte[] cf1 = Bytes.toBytes("CF1");
6702     byte[][] families = { cf1 };
6703     final int SLEEP_TIME = 10 * 1000;
6704 
6705     Configuration conf = new Configuration(CONF);
6706     // Enable close thread interrupt and server abort behavior.
6707     conf.setBoolean(HRegion.CLOSE_WAIT_ABORT, true);
6708     // Set the abort interval to a fraction of sleep time so we are guaranteed to be aborted.
6709     conf.setInt(HRegion.CLOSE_WAIT_TIME, SLEEP_TIME / 2);
6710     // Set the wait interval to a fraction of sleep time so we are guaranteed to be interrupted.
6711     conf.setInt(HRegion.CLOSE_WAIT_INTERVAL, SLEEP_TIME / 4);
6712     region = initHRegion(tableName, method, conf, families);
6713     RegionServerServices rsServices = mock(RegionServerServices.class);
6714     when(rsServices.getServerName()).thenReturn(ServerName.valueOf("localhost", 1000, 1000));
6715     region.rsServices = rsServices;
6716 
6717     final CountDownLatch latch = new CountDownLatch(1);
6718     Thread holder = new Thread(new Runnable() {
6719       @Override
6720       public void run() {
6721         try {
6722           LOG.info("Starting region operation holder");
6723           region.startRegionOperation(Operation.SCAN);
6724           latch.countDown();
6725           // Hold the lock for SLEEP_TIME seconds no matter how many times we are interrupted.
6726           int timeRemaining = SLEEP_TIME;
6727           while (timeRemaining > 0) {
6728             long start = EnvironmentEdgeManager.currentTime();
6729             try {
6730               Thread.sleep(timeRemaining);
6731             } catch (InterruptedException e) {
6732               LOG.info("Interrupted");
6733             }
6734             long end = EnvironmentEdgeManager.currentTime();
6735             timeRemaining -= end - start;
6736             if (timeRemaining < 0) {
6737               timeRemaining = 0;
6738             }
6739             if (timeRemaining > 0) {
6740               LOG.info("Sleeping again, remaining time " + timeRemaining + " ms");
6741             }
6742           }
6743         } catch (Exception e) {
6744           throw new RuntimeException(e);
6745         } finally {
6746           try {
6747             region.closeRegionOperation();
6748           } catch (IOException e) {
6749           }
6750           LOG.info("Stopped region operation holder");
6751         }
6752       }
6753     });
6754 
6755     holder.start();
6756     latch.await();
6757     try {
6758       region.close();
6759     } catch (IOException e) {
6760       LOG.info("Caught expected exception", e);
6761     }
6762     region = null;
6763     holder.join();
6764 
6765     // Verify the region tried to abort the server
6766     verify(rsServices, atLeast(1)).abort(anyString(),any(Throwable.class));
6767   }
6768 
6769   @Test
6770   public void testInterruptProtection() throws Exception {
6771     byte[] cf1 = Bytes.toBytes("CF1");
6772     byte[][] families = { cf1 };
6773     final int SLEEP_TIME = 10 * 1000;
6774 
6775     Configuration conf = new Configuration(CONF);
6776     // Enable close thread interrupt and server abort behavior.
6777     conf.setBoolean(HRegion.CLOSE_WAIT_ABORT, true);
6778     conf.setInt(HRegion.CLOSE_WAIT_INTERVAL, 1000);
6779     region = initHRegion(tableName, method, conf, families);
6780 
6781     final CountDownLatch latch = new CountDownLatch(1);
6782     final AtomicBoolean holderInterrupted = new AtomicBoolean();
6783     Thread holder = new Thread(new Runnable() {
6784       @Override
6785       public void run() {
6786         try {
6787           LOG.info("Starting region operation holder");
6788           region.startRegionOperation(Operation.SCAN);
6789           LOG.info("Protecting against interrupts");
6790           region.disableInterrupts();
6791           try {
6792             latch.countDown();
6793             try {
6794               Thread.sleep(SLEEP_TIME);
6795             } catch (InterruptedException e) {
6796               LOG.info("Interrupted");
6797               holderInterrupted.set(true);
6798             }
6799           } finally {
6800             region.enableInterrupts();
6801           }
6802         } catch (Exception e) {
6803           throw new RuntimeException(e);
6804         } finally {
6805           try {
6806             region.closeRegionOperation();
6807           } catch (IOException e) {
6808           }
6809           LOG.info("Stopped region operation holder");
6810         }
6811       }
6812     });
6813 
6814     holder.start();
6815     latch.await();
6816     region.close();
6817     region = null;
6818     holder.join();
6819 
6820     assertFalse("Region lock holder should not have been interrupted", holderInterrupted.get());
6821   }
6822 
6823 }