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