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