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