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.junit.Assert.assertArrayEquals;
021import static org.junit.Assert.assertEquals;
022import static org.junit.Assert.assertFalse;
023import static org.junit.Assert.assertNull;
024import static org.junit.Assert.assertTrue;
025import static org.junit.Assert.fail;
026import static org.mockito.ArgumentMatchers.any;
027import static org.mockito.Mockito.mock;
028import static org.mockito.Mockito.spy;
029import static org.mockito.Mockito.times;
030import static org.mockito.Mockito.verify;
031import static org.mockito.Mockito.when;
032
033import java.io.IOException;
034import java.lang.ref.SoftReference;
035import java.security.PrivilegedExceptionAction;
036import java.util.ArrayList;
037import java.util.Arrays;
038import java.util.Collection;
039import java.util.Collections;
040import java.util.Iterator;
041import java.util.List;
042import java.util.ListIterator;
043import java.util.NavigableSet;
044import java.util.TreeSet;
045import java.util.concurrent.ConcurrentSkipListSet;
046import java.util.concurrent.CountDownLatch;
047import java.util.concurrent.CyclicBarrier;
048import java.util.concurrent.ExecutorService;
049import java.util.concurrent.Executors;
050import java.util.concurrent.ThreadPoolExecutor;
051import java.util.concurrent.TimeUnit;
052import java.util.concurrent.atomic.AtomicBoolean;
053import java.util.concurrent.atomic.AtomicInteger;
054import java.util.concurrent.atomic.AtomicLong;
055import java.util.concurrent.atomic.AtomicReference;
056import java.util.concurrent.locks.ReentrantReadWriteLock;
057import java.util.function.Consumer;
058import java.util.function.IntBinaryOperator;
059import org.apache.hadoop.conf.Configuration;
060import org.apache.hadoop.fs.FSDataOutputStream;
061import org.apache.hadoop.fs.FileStatus;
062import org.apache.hadoop.fs.FileSystem;
063import org.apache.hadoop.fs.FilterFileSystem;
064import org.apache.hadoop.fs.LocalFileSystem;
065import org.apache.hadoop.fs.Path;
066import org.apache.hadoop.fs.permission.FsPermission;
067import org.apache.hadoop.hbase.Cell;
068import org.apache.hadoop.hbase.CellBuilderFactory;
069import org.apache.hadoop.hbase.CellBuilderType;
070import org.apache.hadoop.hbase.CellComparator;
071import org.apache.hadoop.hbase.CellComparatorImpl;
072import org.apache.hadoop.hbase.CellUtil;
073import org.apache.hadoop.hbase.HBaseClassTestRule;
074import org.apache.hadoop.hbase.HBaseConfiguration;
075import org.apache.hadoop.hbase.HBaseTestingUtil;
076import org.apache.hadoop.hbase.HConstants;
077import org.apache.hadoop.hbase.KeyValue;
078import org.apache.hadoop.hbase.MemoryCompactionPolicy;
079import org.apache.hadoop.hbase.PrivateCellUtil;
080import org.apache.hadoop.hbase.TableName;
081import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
082import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
083import org.apache.hadoop.hbase.client.Get;
084import org.apache.hadoop.hbase.client.RegionInfo;
085import org.apache.hadoop.hbase.client.RegionInfoBuilder;
086import org.apache.hadoop.hbase.client.Scan;
087import org.apache.hadoop.hbase.client.Scan.ReadType;
088import org.apache.hadoop.hbase.client.TableDescriptor;
089import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
090import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;
091import org.apache.hadoop.hbase.filter.Filter;
092import org.apache.hadoop.hbase.filter.FilterBase;
093import org.apache.hadoop.hbase.io.compress.Compression;
094import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
095import org.apache.hadoop.hbase.io.hfile.CacheConfig;
096import org.apache.hadoop.hbase.io.hfile.HFile;
097import org.apache.hadoop.hbase.io.hfile.HFileContext;
098import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
099import org.apache.hadoop.hbase.monitoring.MonitoredTask;
100import org.apache.hadoop.hbase.nio.RefCnt;
101import org.apache.hadoop.hbase.quotas.RegionSizeStoreImpl;
102import org.apache.hadoop.hbase.regionserver.MemStoreCompactionStrategy.Action;
103import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
104import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
105import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
106import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
107import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
108import org.apache.hadoop.hbase.security.User;
109import org.apache.hadoop.hbase.testclassification.MediumTests;
110import org.apache.hadoop.hbase.testclassification.RegionServerTests;
111import org.apache.hadoop.hbase.util.Bytes;
112import org.apache.hadoop.hbase.util.CommonFSUtils;
113import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
114import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
115import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
116import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
117import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
118import org.apache.hadoop.hbase.wal.WALFactory;
119import org.apache.hadoop.util.Progressable;
120import org.junit.After;
121import org.junit.AfterClass;
122import org.junit.Before;
123import org.junit.ClassRule;
124import org.junit.Rule;
125import org.junit.Test;
126import org.junit.experimental.categories.Category;
127import org.junit.rules.TestName;
128import org.mockito.Mockito;
129import org.slf4j.Logger;
130import org.slf4j.LoggerFactory;
131
132import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
133
134/**
135 * Test class for the HStore
136 */
137@Category({ RegionServerTests.class, MediumTests.class })
138public class TestHStore {
139
140  @ClassRule
141  public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestHStore.class);
142
143  private static final Logger LOG = LoggerFactory.getLogger(TestHStore.class);
144  @Rule
145  public TestName name = new TestName();
146
147  HRegion region;
148  HStore store;
149  byte[] table = Bytes.toBytes("table");
150  byte[] family = Bytes.toBytes("family");
151
152  byte[] row = Bytes.toBytes("row");
153  byte[] row2 = Bytes.toBytes("row2");
154  byte[] qf1 = Bytes.toBytes("qf1");
155  byte[] qf2 = Bytes.toBytes("qf2");
156  byte[] qf3 = Bytes.toBytes("qf3");
157  byte[] qf4 = Bytes.toBytes("qf4");
158  byte[] qf5 = Bytes.toBytes("qf5");
159  byte[] qf6 = Bytes.toBytes("qf6");
160
161  NavigableSet<byte[]> qualifiers = new ConcurrentSkipListSet<>(Bytes.BYTES_COMPARATOR);
162
163  List<Cell> expected = new ArrayList<>();
164  List<Cell> result = new ArrayList<>();
165
166  long id = EnvironmentEdgeManager.currentTime();
167  Get get = new Get(row);
168
169  private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
170  private static final String DIR = TEST_UTIL.getDataTestDir("TestStore").toString();
171
172  @Before
173  public void setUp() throws IOException {
174    qualifiers.clear();
175    qualifiers.add(qf1);
176    qualifiers.add(qf3);
177    qualifiers.add(qf5);
178
179    Iterator<byte[]> iter = qualifiers.iterator();
180    while (iter.hasNext()) {
181      byte[] next = iter.next();
182      expected.add(new KeyValue(row, family, next, 1, (byte[]) null));
183      get.addColumn(family, next);
184    }
185  }
186
187  private void init(String methodName) throws IOException {
188    init(methodName, TEST_UTIL.getConfiguration());
189  }
190
191  private HStore init(String methodName, Configuration conf) throws IOException {
192    // some of the tests write 4 versions and then flush
193    // (with HBASE-4241, lower versions are collected on flush)
194    return init(methodName, conf,
195      ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(4).build());
196  }
197
198  private HStore init(String methodName, Configuration conf, ColumnFamilyDescriptor hcd)
199    throws IOException {
200    return init(methodName, conf, TableDescriptorBuilder.newBuilder(TableName.valueOf(table)), hcd);
201  }
202
203  private HStore init(String methodName, Configuration conf, TableDescriptorBuilder builder,
204    ColumnFamilyDescriptor hcd) throws IOException {
205    return init(methodName, conf, builder, hcd, null);
206  }
207
208  private HStore init(String methodName, Configuration conf, TableDescriptorBuilder builder,
209    ColumnFamilyDescriptor hcd, MyStoreHook hook) throws IOException {
210    return init(methodName, conf, builder, hcd, hook, false);
211  }
212
213  private void initHRegion(String methodName, Configuration conf, TableDescriptorBuilder builder,
214    ColumnFamilyDescriptor hcd, MyStoreHook hook, boolean switchToPread) throws IOException {
215    TableDescriptor htd = builder.setColumnFamily(hcd).build();
216    Path basedir = new Path(DIR + methodName);
217    Path tableDir = CommonFSUtils.getTableDir(basedir, htd.getTableName());
218    final Path logdir = new Path(basedir, AbstractFSWALProvider.getWALDirectoryName(methodName));
219
220    FileSystem fs = FileSystem.get(conf);
221
222    fs.delete(logdir, true);
223    ChunkCreator.initialize(MemStoreLAB.CHUNK_SIZE_DEFAULT, false,
224      MemStoreLABImpl.CHUNK_SIZE_DEFAULT, 1, 0, null,
225      MemStoreLAB.INDEX_CHUNK_SIZE_PERCENTAGE_DEFAULT);
226    RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).build();
227    Configuration walConf = new Configuration(conf);
228    CommonFSUtils.setRootDir(walConf, basedir);
229    WALFactory wals = new WALFactory(walConf, methodName);
230    region = new HRegion(new HRegionFileSystem(conf, fs, tableDir, info), wals.getWAL(info), conf,
231      htd, null);
232    region.regionServicesForStores = Mockito.spy(region.regionServicesForStores);
233    ThreadPoolExecutor pool = (ThreadPoolExecutor) Executors.newFixedThreadPool(1);
234    Mockito.when(region.regionServicesForStores.getInMemoryCompactionPool()).thenReturn(pool);
235  }
236
237  private HStore init(String methodName, Configuration conf, TableDescriptorBuilder builder,
238    ColumnFamilyDescriptor hcd, MyStoreHook hook, boolean switchToPread) throws IOException {
239    initHRegion(methodName, conf, builder, hcd, hook, switchToPread);
240    if (hook == null) {
241      store = new HStore(region, hcd, conf, false);
242    } else {
243      store = new MyStore(region, hcd, conf, hook, switchToPread);
244    }
245    region.stores.put(store.getColumnFamilyDescriptor().getName(), store);
246    return store;
247  }
248
249  /**
250   * Test we do not lose data if we fail a flush and then close. Part of HBase-10466
251   */
252  @Test
253  public void testFlushSizeSizing() throws Exception {
254    LOG.info("Setting up a faulty file system that cannot write in " + this.name.getMethodName());
255    final Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
256    // Only retry once.
257    conf.setInt("hbase.hstore.flush.retries.number", 1);
258    User user = User.createUserForTesting(conf, this.name.getMethodName(), new String[] { "foo" });
259    // Inject our faulty LocalFileSystem
260    conf.setClass("fs.file.impl", FaultyFileSystem.class, FileSystem.class);
261    user.runAs(new PrivilegedExceptionAction<Object>() {
262      @Override
263      public Object run() throws Exception {
264        // Make sure it worked (above is sensitive to caching details in hadoop core)
265        FileSystem fs = FileSystem.get(conf);
266        assertEquals(FaultyFileSystem.class, fs.getClass());
267        FaultyFileSystem ffs = (FaultyFileSystem) fs;
268
269        // Initialize region
270        init(name.getMethodName(), conf);
271
272        MemStoreSize mss = store.memstore.getFlushableSize();
273        assertEquals(0, mss.getDataSize());
274        LOG.info("Adding some data");
275        MemStoreSizing kvSize = new NonThreadSafeMemStoreSizing();
276        store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), kvSize);
277        // add the heap size of active (mutable) segment
278        kvSize.incMemStoreSize(0, MutableSegment.DEEP_OVERHEAD, 0, 0);
279        mss = store.memstore.getFlushableSize();
280        assertEquals(kvSize.getMemStoreSize(), mss);
281        // Flush. Bug #1 from HBASE-10466. Make sure size calculation on failed flush is right.
282        try {
283          LOG.info("Flushing");
284          flushStore(store, id++);
285          fail("Didn't bubble up IOE!");
286        } catch (IOException ioe) {
287          assertTrue(ioe.getMessage().contains("Fault injected"));
288        }
289        // due to snapshot, change mutable to immutable segment
290        kvSize.incMemStoreSize(0,
291          CSLMImmutableSegment.DEEP_OVERHEAD_CSLM - MutableSegment.DEEP_OVERHEAD, 0, 0);
292        mss = store.memstore.getFlushableSize();
293        assertEquals(kvSize.getMemStoreSize(), mss);
294        MemStoreSizing kvSize2 = new NonThreadSafeMemStoreSizing();
295        store.add(new KeyValue(row, family, qf2, 2, (byte[]) null), kvSize2);
296        kvSize2.incMemStoreSize(0, MutableSegment.DEEP_OVERHEAD, 0, 0);
297        // Even though we add a new kv, we expect the flushable size to be 'same' since we have
298        // not yet cleared the snapshot -- the above flush failed.
299        assertEquals(kvSize.getMemStoreSize(), mss);
300        ffs.fault.set(false);
301        flushStore(store, id++);
302        mss = store.memstore.getFlushableSize();
303        // Size should be the foreground kv size.
304        assertEquals(kvSize2.getMemStoreSize(), mss);
305        flushStore(store, id++);
306        mss = store.memstore.getFlushableSize();
307        assertEquals(0, mss.getDataSize());
308        assertEquals(MutableSegment.DEEP_OVERHEAD, mss.getHeapSize());
309        return null;
310      }
311    });
312  }
313
314  /**
315   * Verify that compression and data block encoding are respected by the createWriter method, used
316   * on store flush.
317   */
318  @Test
319  public void testCreateWriter() throws Exception {
320    Configuration conf = HBaseConfiguration.create();
321    FileSystem fs = FileSystem.get(conf);
322
323    ColumnFamilyDescriptor hcd =
324      ColumnFamilyDescriptorBuilder.newBuilder(family).setCompressionType(Compression.Algorithm.GZ)
325        .setDataBlockEncoding(DataBlockEncoding.DIFF).build();
326    init(name.getMethodName(), conf, hcd);
327
328    // Test createWriter
329    StoreFileWriter writer = store.getStoreEngine()
330      .createWriter(CreateStoreFileWriterParams.create().maxKeyCount(4)
331        .compression(hcd.getCompressionType()).isCompaction(false).includeMVCCReadpoint(true)
332        .includesTag(false).shouldDropBehind(false));
333    Path path = writer.getPath();
334    writer.append(new KeyValue(row, family, qf1, Bytes.toBytes(1)));
335    writer.append(new KeyValue(row, family, qf2, Bytes.toBytes(2)));
336    writer.append(new KeyValue(row2, family, qf1, Bytes.toBytes(3)));
337    writer.append(new KeyValue(row2, family, qf2, Bytes.toBytes(4)));
338    writer.close();
339
340    // Verify that compression and encoding settings are respected
341    HFile.Reader reader = HFile.createReader(fs, path, new CacheConfig(conf), true, conf);
342    assertEquals(hcd.getCompressionType(), reader.getTrailer().getCompressionCodec());
343    assertEquals(hcd.getDataBlockEncoding(), reader.getDataBlockEncoding());
344    reader.close();
345  }
346
347  @Test
348  public void testDeleteExpiredStoreFiles() throws Exception {
349    testDeleteExpiredStoreFiles(0);
350    testDeleteExpiredStoreFiles(1);
351  }
352
353  /**
354   * @param minVersions the MIN_VERSIONS for the column family
355   */
356  public void testDeleteExpiredStoreFiles(int minVersions) throws Exception {
357    int storeFileNum = 4;
358    int ttl = 4;
359    IncrementingEnvironmentEdge edge = new IncrementingEnvironmentEdge();
360    EnvironmentEdgeManagerTestHelper.injectEdge(edge);
361
362    Configuration conf = HBaseConfiguration.create();
363    // Enable the expired store file deletion
364    conf.setBoolean("hbase.store.delete.expired.storefile", true);
365    // Set the compaction threshold higher to avoid normal compactions.
366    conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MIN_KEY, 5);
367
368    init(name.getMethodName() + "-" + minVersions, conf, ColumnFamilyDescriptorBuilder
369      .newBuilder(family).setMinVersions(minVersions).setTimeToLive(ttl).build());
370
371    long storeTtl = this.store.getScanInfo().getTtl();
372    long sleepTime = storeTtl / storeFileNum;
373    long timeStamp;
374    // There are 4 store files and the max time stamp difference among these
375    // store files will be (this.store.ttl / storeFileNum)
376    for (int i = 1; i <= storeFileNum; i++) {
377      LOG.info("Adding some data for the store file #" + i);
378      timeStamp = EnvironmentEdgeManager.currentTime();
379      this.store.add(new KeyValue(row, family, qf1, timeStamp, (byte[]) null), null);
380      this.store.add(new KeyValue(row, family, qf2, timeStamp, (byte[]) null), null);
381      this.store.add(new KeyValue(row, family, qf3, timeStamp, (byte[]) null), null);
382      flush(i);
383      edge.incrementTime(sleepTime);
384    }
385
386    // Verify the total number of store files
387    assertEquals(storeFileNum, this.store.getStorefiles().size());
388
389    // Each call will find one expired store file and delete it before compaction happens.
390    // There will be no compaction due to threshold above. Last file will not be replaced.
391    for (int i = 1; i <= storeFileNum - 1; i++) {
392      // verify the expired store file.
393      assertFalse(this.store.requestCompaction().isPresent());
394      Collection<HStoreFile> sfs = this.store.getStorefiles();
395      // Ensure i files are gone.
396      if (minVersions == 0) {
397        assertEquals(storeFileNum - i, sfs.size());
398        // Ensure only non-expired files remain.
399        for (HStoreFile sf : sfs) {
400          assertTrue(sf.getReader().getMaxTimestamp() >= (edge.currentTime() - storeTtl));
401        }
402      } else {
403        assertEquals(storeFileNum, sfs.size());
404      }
405      // Let the next store file expired.
406      edge.incrementTime(sleepTime);
407    }
408    assertFalse(this.store.requestCompaction().isPresent());
409
410    Collection<HStoreFile> sfs = this.store.getStorefiles();
411    // Assert the last expired file is not removed.
412    if (minVersions == 0) {
413      assertEquals(1, sfs.size());
414    }
415    long ts = sfs.iterator().next().getReader().getMaxTimestamp();
416    assertTrue(ts < (edge.currentTime() - storeTtl));
417
418    for (HStoreFile sf : sfs) {
419      sf.closeStoreFile(true);
420    }
421  }
422
423  @Test
424  public void testLowestModificationTime() throws Exception {
425    Configuration conf = HBaseConfiguration.create();
426    FileSystem fs = FileSystem.get(conf);
427    // Initialize region
428    init(name.getMethodName(), conf);
429
430    int storeFileNum = 4;
431    for (int i = 1; i <= storeFileNum; i++) {
432      LOG.info("Adding some data for the store file #" + i);
433      this.store.add(new KeyValue(row, family, qf1, i, (byte[]) null), null);
434      this.store.add(new KeyValue(row, family, qf2, i, (byte[]) null), null);
435      this.store.add(new KeyValue(row, family, qf3, i, (byte[]) null), null);
436      flush(i);
437    }
438    // after flush; check the lowest time stamp
439    long lowestTimeStampFromManager = StoreUtils.getLowestTimestamp(store.getStorefiles());
440    long lowestTimeStampFromFS = getLowestTimeStampFromFS(fs, store.getStorefiles());
441    assertEquals(lowestTimeStampFromManager, lowestTimeStampFromFS);
442
443    // after compact; check the lowest time stamp
444    store.compact(store.requestCompaction().get(), NoLimitThroughputController.INSTANCE, null);
445    lowestTimeStampFromManager = StoreUtils.getLowestTimestamp(store.getStorefiles());
446    lowestTimeStampFromFS = getLowestTimeStampFromFS(fs, store.getStorefiles());
447    assertEquals(lowestTimeStampFromManager, lowestTimeStampFromFS);
448  }
449
450  private static long getLowestTimeStampFromFS(FileSystem fs,
451    final Collection<HStoreFile> candidates) throws IOException {
452    long minTs = Long.MAX_VALUE;
453    if (candidates.isEmpty()) {
454      return minTs;
455    }
456    Path[] p = new Path[candidates.size()];
457    int i = 0;
458    for (HStoreFile sf : candidates) {
459      p[i] = sf.getPath();
460      ++i;
461    }
462
463    FileStatus[] stats = fs.listStatus(p);
464    if (stats == null || stats.length == 0) {
465      return minTs;
466    }
467    for (FileStatus s : stats) {
468      minTs = Math.min(minTs, s.getModificationTime());
469    }
470    return minTs;
471  }
472
473  //////////////////////////////////////////////////////////////////////////////
474  // Get tests
475  //////////////////////////////////////////////////////////////////////////////
476
477  private static final int BLOCKSIZE_SMALL = 8192;
478
479  /**
480   * Test for hbase-1686.
481   */
482  @Test
483  public void testEmptyStoreFile() throws IOException {
484    init(this.name.getMethodName());
485    // Write a store file.
486    this.store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), null);
487    this.store.add(new KeyValue(row, family, qf2, 1, (byte[]) null), null);
488    flush(1);
489    // Now put in place an empty store file. Its a little tricky. Have to
490    // do manually with hacked in sequence id.
491    HStoreFile f = this.store.getStorefiles().iterator().next();
492    Path storedir = f.getPath().getParent();
493    long seqid = f.getMaxSequenceId();
494    Configuration c = HBaseConfiguration.create();
495    FileSystem fs = FileSystem.get(c);
496    HFileContext meta = new HFileContextBuilder().withBlockSize(BLOCKSIZE_SMALL).build();
497    StoreFileWriter w = new StoreFileWriter.Builder(c, new CacheConfig(c), fs)
498      .withOutputDir(storedir).withFileContext(meta).build();
499    w.appendMetadata(seqid + 1, false);
500    w.close();
501    this.store.close();
502    // Reopen it... should pick up two files
503    this.store =
504      new HStore(this.store.getHRegion(), this.store.getColumnFamilyDescriptor(), c, false);
505    assertEquals(2, this.store.getStorefilesCount());
506
507    result = HBaseTestingUtil.getFromStoreFile(store, get.getRow(), qualifiers);
508    assertEquals(1, result.size());
509  }
510
511  /**
512   * Getting data from memstore only
513   */
514  @Test
515  public void testGet_FromMemStoreOnly() throws IOException {
516    init(this.name.getMethodName());
517
518    // Put data in memstore
519    this.store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), null);
520    this.store.add(new KeyValue(row, family, qf2, 1, (byte[]) null), null);
521    this.store.add(new KeyValue(row, family, qf3, 1, (byte[]) null), null);
522    this.store.add(new KeyValue(row, family, qf4, 1, (byte[]) null), null);
523    this.store.add(new KeyValue(row, family, qf5, 1, (byte[]) null), null);
524    this.store.add(new KeyValue(row, family, qf6, 1, (byte[]) null), null);
525
526    // Get
527    result = HBaseTestingUtil.getFromStoreFile(store, get.getRow(), qualifiers);
528
529    // Compare
530    assertCheck();
531  }
532
533  @Test
534  public void testTimeRangeIfSomeCellsAreDroppedInFlush() throws IOException {
535    testTimeRangeIfSomeCellsAreDroppedInFlush(1);
536    testTimeRangeIfSomeCellsAreDroppedInFlush(3);
537    testTimeRangeIfSomeCellsAreDroppedInFlush(5);
538  }
539
540  private void testTimeRangeIfSomeCellsAreDroppedInFlush(int maxVersion) throws IOException {
541    init(this.name.getMethodName(), TEST_UTIL.getConfiguration(),
542      ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(maxVersion).build());
543    long currentTs = 100;
544    long minTs = currentTs;
545    // the extra cell won't be flushed to disk,
546    // so the min of timerange will be different between memStore and hfile.
547    for (int i = 0; i != (maxVersion + 1); ++i) {
548      this.store.add(new KeyValue(row, family, qf1, ++currentTs, (byte[]) null), null);
549      if (i == 1) {
550        minTs = currentTs;
551      }
552    }
553    flushStore(store, id++);
554
555    Collection<HStoreFile> files = store.getStorefiles();
556    assertEquals(1, files.size());
557    HStoreFile f = files.iterator().next();
558    f.initReader();
559    StoreFileReader reader = f.getReader();
560    assertEquals(minTs, reader.timeRange.getMin());
561    assertEquals(currentTs, reader.timeRange.getMax());
562  }
563
564  /**
565   * Getting data from files only
566   */
567  @Test
568  public void testGet_FromFilesOnly() throws IOException {
569    init(this.name.getMethodName());
570
571    // Put data in memstore
572    this.store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), null);
573    this.store.add(new KeyValue(row, family, qf2, 1, (byte[]) null), null);
574    // flush
575    flush(1);
576
577    // Add more data
578    this.store.add(new KeyValue(row, family, qf3, 1, (byte[]) null), null);
579    this.store.add(new KeyValue(row, family, qf4, 1, (byte[]) null), null);
580    // flush
581    flush(2);
582
583    // Add more data
584    this.store.add(new KeyValue(row, family, qf5, 1, (byte[]) null), null);
585    this.store.add(new KeyValue(row, family, qf6, 1, (byte[]) null), null);
586    // flush
587    flush(3);
588
589    // Get
590    result = HBaseTestingUtil.getFromStoreFile(store, get.getRow(), qualifiers);
591    // this.store.get(get, qualifiers, result);
592
593    // Need to sort the result since multiple files
594    Collections.sort(result, CellComparatorImpl.COMPARATOR);
595
596    // Compare
597    assertCheck();
598  }
599
600  /**
601   * Getting data from memstore and files
602   */
603  @Test
604  public void testGet_FromMemStoreAndFiles() throws IOException {
605    init(this.name.getMethodName());
606
607    // Put data in memstore
608    this.store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), null);
609    this.store.add(new KeyValue(row, family, qf2, 1, (byte[]) null), null);
610    // flush
611    flush(1);
612
613    // Add more data
614    this.store.add(new KeyValue(row, family, qf3, 1, (byte[]) null), null);
615    this.store.add(new KeyValue(row, family, qf4, 1, (byte[]) null), null);
616    // flush
617    flush(2);
618
619    // Add more data
620    this.store.add(new KeyValue(row, family, qf5, 1, (byte[]) null), null);
621    this.store.add(new KeyValue(row, family, qf6, 1, (byte[]) null), null);
622
623    // Get
624    result = HBaseTestingUtil.getFromStoreFile(store, get.getRow(), qualifiers);
625
626    // Need to sort the result since multiple files
627    Collections.sort(result, CellComparatorImpl.COMPARATOR);
628
629    // Compare
630    assertCheck();
631  }
632
633  private void flush(int storeFilessize) throws IOException {
634    flushStore(store, id++);
635    assertEquals(storeFilessize, this.store.getStorefiles().size());
636    assertEquals(0, ((AbstractMemStore) this.store.memstore).getActive().getCellsCount());
637  }
638
639  private void assertCheck() {
640    assertEquals(expected.size(), result.size());
641    for (int i = 0; i < expected.size(); i++) {
642      assertEquals(expected.get(i), result.get(i));
643    }
644  }
645
646  @After
647  public void tearDown() throws Exception {
648    EnvironmentEdgeManagerTestHelper.reset();
649    if (store != null) {
650      try {
651        store.close();
652      } catch (IOException e) {
653      }
654      store = null;
655    }
656    if (region != null) {
657      region.close();
658      region = null;
659    }
660  }
661
662  @AfterClass
663  public static void tearDownAfterClass() throws IOException {
664    TEST_UTIL.cleanupTestDir();
665  }
666
667  @Test
668  public void testHandleErrorsInFlush() throws Exception {
669    LOG.info("Setting up a faulty file system that cannot write");
670
671    final Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
672    User user = User.createUserForTesting(conf, "testhandleerrorsinflush", new String[] { "foo" });
673    // Inject our faulty LocalFileSystem
674    conf.setClass("fs.file.impl", FaultyFileSystem.class, FileSystem.class);
675    user.runAs(new PrivilegedExceptionAction<Object>() {
676      @Override
677      public Object run() throws Exception {
678        // Make sure it worked (above is sensitive to caching details in hadoop core)
679        FileSystem fs = FileSystem.get(conf);
680        assertEquals(FaultyFileSystem.class, fs.getClass());
681
682        // Initialize region
683        init(name.getMethodName(), conf);
684
685        LOG.info("Adding some data");
686        store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), null);
687        store.add(new KeyValue(row, family, qf2, 1, (byte[]) null), null);
688        store.add(new KeyValue(row, family, qf3, 1, (byte[]) null), null);
689
690        LOG.info("Before flush, we should have no files");
691
692        Collection<StoreFileInfo> files =
693          store.getRegionFileSystem().getStoreFiles(store.getColumnFamilyName());
694        assertEquals(0, files != null ? files.size() : 0);
695
696        // flush
697        try {
698          LOG.info("Flushing");
699          flush(1);
700          fail("Didn't bubble up IOE!");
701        } catch (IOException ioe) {
702          assertTrue(ioe.getMessage().contains("Fault injected"));
703        }
704
705        LOG.info("After failed flush, we should still have no files!");
706        files = store.getRegionFileSystem().getStoreFiles(store.getColumnFamilyName());
707        assertEquals(0, files != null ? files.size() : 0);
708        store.getHRegion().getWAL().close();
709        return null;
710      }
711    });
712    FileSystem.closeAllForUGI(user.getUGI());
713  }
714
715  /**
716   * Faulty file system that will fail if you write past its fault position the FIRST TIME only;
717   * thereafter it will succeed. Used by {@link TestHRegion} too.
718   */
719  static class FaultyFileSystem extends FilterFileSystem {
720    List<SoftReference<FaultyOutputStream>> outStreams = new ArrayList<>();
721    private long faultPos = 200;
722    AtomicBoolean fault = new AtomicBoolean(true);
723
724    public FaultyFileSystem() {
725      super(new LocalFileSystem());
726      LOG.info("Creating faulty!");
727    }
728
729    @Override
730    public FSDataOutputStream create(Path p) throws IOException {
731      return new FaultyOutputStream(super.create(p), faultPos, fault);
732    }
733
734    @Override
735    public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite,
736      int bufferSize, short replication, long blockSize, Progressable progress) throws IOException {
737      return new FaultyOutputStream(
738        super.create(f, permission, overwrite, bufferSize, replication, blockSize, progress),
739        faultPos, fault);
740    }
741
742    @Override
743    public FSDataOutputStream createNonRecursive(Path f, boolean overwrite, int bufferSize,
744      short replication, long blockSize, Progressable progress) throws IOException {
745      // Fake it. Call create instead. The default implementation throws an IOE
746      // that this is not supported.
747      return create(f, overwrite, bufferSize, replication, blockSize, progress);
748    }
749  }
750
751  static class FaultyOutputStream extends FSDataOutputStream {
752    volatile long faultPos = Long.MAX_VALUE;
753    private final AtomicBoolean fault;
754
755    public FaultyOutputStream(FSDataOutputStream out, long faultPos, final AtomicBoolean fault)
756      throws IOException {
757      super(out, null);
758      this.faultPos = faultPos;
759      this.fault = fault;
760    }
761
762    @Override
763    public synchronized void write(byte[] buf, int offset, int length) throws IOException {
764      LOG.info("faulty stream write at pos " + getPos());
765      injectFault();
766      super.write(buf, offset, length);
767    }
768
769    private void injectFault() throws IOException {
770      if (this.fault.get() && getPos() >= faultPos) {
771        throw new IOException("Fault injected");
772      }
773    }
774  }
775
776  private static StoreFlushContext flushStore(HStore store, long id) throws IOException {
777    StoreFlushContext storeFlushCtx = store.createFlushContext(id, FlushLifeCycleTracker.DUMMY);
778    storeFlushCtx.prepare();
779    storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));
780    storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));
781    return storeFlushCtx;
782  }
783
784  /**
785   * Generate a list of KeyValues for testing based on given parameters
786   * @return the rows key-value list
787   */
788  private List<Cell> getKeyValueSet(long[] timestamps, int numRows, byte[] qualifier,
789    byte[] family) {
790    List<Cell> kvList = new ArrayList<>();
791    for (int i = 1; i <= numRows; i++) {
792      byte[] b = Bytes.toBytes(i);
793      for (long timestamp : timestamps) {
794        kvList.add(new KeyValue(b, family, qualifier, timestamp, b));
795      }
796    }
797    return kvList;
798  }
799
800  /**
801   * Test to ensure correctness when using Stores with multiple timestamps
802   */
803  @Test
804  public void testMultipleTimestamps() throws IOException {
805    int numRows = 1;
806    long[] timestamps1 = new long[] { 1, 5, 10, 20 };
807    long[] timestamps2 = new long[] { 30, 80 };
808
809    init(this.name.getMethodName());
810
811    List<Cell> kvList1 = getKeyValueSet(timestamps1, numRows, qf1, family);
812    for (Cell kv : kvList1) {
813      this.store.add(kv, null);
814    }
815
816    flushStore(store, id++);
817
818    List<Cell> kvList2 = getKeyValueSet(timestamps2, numRows, qf1, family);
819    for (Cell kv : kvList2) {
820      this.store.add(kv, null);
821    }
822
823    List<Cell> result;
824    Get get = new Get(Bytes.toBytes(1));
825    get.addColumn(family, qf1);
826
827    get.setTimeRange(0, 15);
828    result = HBaseTestingUtil.getFromStoreFile(store, get);
829    assertTrue(result.size() > 0);
830
831    get.setTimeRange(40, 90);
832    result = HBaseTestingUtil.getFromStoreFile(store, get);
833    assertTrue(result.size() > 0);
834
835    get.setTimeRange(10, 45);
836    result = HBaseTestingUtil.getFromStoreFile(store, get);
837    assertTrue(result.size() > 0);
838
839    get.setTimeRange(80, 145);
840    result = HBaseTestingUtil.getFromStoreFile(store, get);
841    assertTrue(result.size() > 0);
842
843    get.setTimeRange(1, 2);
844    result = HBaseTestingUtil.getFromStoreFile(store, get);
845    assertTrue(result.size() > 0);
846
847    get.setTimeRange(90, 200);
848    result = HBaseTestingUtil.getFromStoreFile(store, get);
849    assertTrue(result.size() == 0);
850  }
851
852  /**
853   * Test for HBASE-3492 - Test split on empty colfam (no store files).
854   * @throws IOException When the IO operations fail.
855   */
856  @Test
857  public void testSplitWithEmptyColFam() throws IOException {
858    init(this.name.getMethodName());
859    assertFalse(store.getSplitPoint().isPresent());
860  }
861
862  @Test
863  public void testStoreUsesConfigurationFromHcdAndHtd() throws Exception {
864    final String CONFIG_KEY = "hbase.regionserver.thread.compaction.throttle";
865    long anyValue = 10;
866
867    // We'll check that it uses correct config and propagates it appropriately by going thru
868    // the simplest "real" path I can find - "throttleCompaction", which just checks whether
869    // a number we pass in is higher than some config value, inside compactionPolicy.
870    Configuration conf = HBaseConfiguration.create();
871    conf.setLong(CONFIG_KEY, anyValue);
872    init(name.getMethodName() + "-xml", conf);
873    assertTrue(store.throttleCompaction(anyValue + 1));
874    assertFalse(store.throttleCompaction(anyValue));
875
876    // HTD overrides XML.
877    --anyValue;
878    init(
879      name.getMethodName() + "-htd", conf, TableDescriptorBuilder
880        .newBuilder(TableName.valueOf(table)).setValue(CONFIG_KEY, Long.toString(anyValue)),
881      ColumnFamilyDescriptorBuilder.of(family));
882    assertTrue(store.throttleCompaction(anyValue + 1));
883    assertFalse(store.throttleCompaction(anyValue));
884
885    // HCD overrides them both.
886    --anyValue;
887    init(name.getMethodName() + "-hcd", conf,
888      TableDescriptorBuilder.newBuilder(TableName.valueOf(table)).setValue(CONFIG_KEY,
889        Long.toString(anyValue)),
890      ColumnFamilyDescriptorBuilder.newBuilder(family).setValue(CONFIG_KEY, Long.toString(anyValue))
891        .build());
892    assertTrue(store.throttleCompaction(anyValue + 1));
893    assertFalse(store.throttleCompaction(anyValue));
894  }
895
896  public static class DummyStoreEngine extends DefaultStoreEngine {
897    public static DefaultCompactor lastCreatedCompactor = null;
898
899    @Override
900    protected void createComponents(Configuration conf, HStore store, CellComparator comparator)
901      throws IOException {
902      super.createComponents(conf, store, comparator);
903      lastCreatedCompactor = this.compactor;
904    }
905  }
906
907  @Test
908  public void testStoreUsesSearchEngineOverride() throws Exception {
909    Configuration conf = HBaseConfiguration.create();
910    conf.set(StoreEngine.STORE_ENGINE_CLASS_KEY, DummyStoreEngine.class.getName());
911    init(this.name.getMethodName(), conf);
912    assertEquals(DummyStoreEngine.lastCreatedCompactor, this.store.storeEngine.getCompactor());
913  }
914
915  private void addStoreFile() throws IOException {
916    HStoreFile f = this.store.getStorefiles().iterator().next();
917    Path storedir = f.getPath().getParent();
918    long seqid = this.store.getMaxSequenceId().orElse(0L);
919    Configuration c = TEST_UTIL.getConfiguration();
920    FileSystem fs = FileSystem.get(c);
921    HFileContext fileContext = new HFileContextBuilder().withBlockSize(BLOCKSIZE_SMALL).build();
922    StoreFileWriter w = new StoreFileWriter.Builder(c, new CacheConfig(c), fs)
923      .withOutputDir(storedir).withFileContext(fileContext).build();
924    w.appendMetadata(seqid + 1, false);
925    w.close();
926    LOG.info("Added store file:" + w.getPath());
927  }
928
929  private void archiveStoreFile(int index) throws IOException {
930    Collection<HStoreFile> files = this.store.getStorefiles();
931    HStoreFile sf = null;
932    Iterator<HStoreFile> it = files.iterator();
933    for (int i = 0; i <= index; i++) {
934      sf = it.next();
935    }
936    store.getRegionFileSystem().removeStoreFiles(store.getColumnFamilyName(),
937      Lists.newArrayList(sf));
938  }
939
940  private void closeCompactedFile(int index) throws IOException {
941    Collection<HStoreFile> files =
942      this.store.getStoreEngine().getStoreFileManager().getCompactedfiles();
943    if (files.size() > 0) {
944      HStoreFile sf = null;
945      Iterator<HStoreFile> it = files.iterator();
946      for (int i = 0; i <= index; i++) {
947        sf = it.next();
948      }
949      sf.closeStoreFile(true);
950      store.getStoreEngine().getStoreFileManager()
951        .removeCompactedFiles(Collections.singletonList(sf));
952    }
953  }
954
955  @Test
956  public void testRefreshStoreFiles() throws Exception {
957    init(name.getMethodName());
958
959    assertEquals(0, this.store.getStorefilesCount());
960
961    // Test refreshing store files when no store files are there
962    store.refreshStoreFiles();
963    assertEquals(0, this.store.getStorefilesCount());
964
965    // add some data, flush
966    this.store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), null);
967    flush(1);
968    assertEquals(1, this.store.getStorefilesCount());
969
970    // add one more file
971    addStoreFile();
972
973    assertEquals(1, this.store.getStorefilesCount());
974    store.refreshStoreFiles();
975    assertEquals(2, this.store.getStorefilesCount());
976
977    // add three more files
978    addStoreFile();
979    addStoreFile();
980    addStoreFile();
981
982    assertEquals(2, this.store.getStorefilesCount());
983    store.refreshStoreFiles();
984    assertEquals(5, this.store.getStorefilesCount());
985
986    closeCompactedFile(0);
987    archiveStoreFile(0);
988
989    assertEquals(5, this.store.getStorefilesCount());
990    store.refreshStoreFiles();
991    assertEquals(4, this.store.getStorefilesCount());
992
993    archiveStoreFile(0);
994    archiveStoreFile(1);
995    archiveStoreFile(2);
996
997    assertEquals(4, this.store.getStorefilesCount());
998    store.refreshStoreFiles();
999    assertEquals(1, this.store.getStorefilesCount());
1000
1001    archiveStoreFile(0);
1002    store.refreshStoreFiles();
1003    assertEquals(0, this.store.getStorefilesCount());
1004  }
1005
1006  @Test
1007  public void testRefreshStoreFilesNotChanged() throws IOException {
1008    init(name.getMethodName());
1009
1010    assertEquals(0, this.store.getStorefilesCount());
1011
1012    // add some data, flush
1013    this.store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), null);
1014    flush(1);
1015    // add one more file
1016    addStoreFile();
1017
1018    StoreEngine<?, ?, ?, ?> spiedStoreEngine = spy(store.getStoreEngine());
1019
1020    // call first time after files changed
1021    spiedStoreEngine.refreshStoreFiles();
1022    assertEquals(2, this.store.getStorefilesCount());
1023    verify(spiedStoreEngine, times(1)).replaceStoreFiles(any(), any(), any(), any());
1024
1025    // call second time
1026    spiedStoreEngine.refreshStoreFiles();
1027
1028    // ensure that replaceStoreFiles is not called, i.e, the times does not change, if files are not
1029    // refreshed,
1030    verify(spiedStoreEngine, times(1)).replaceStoreFiles(any(), any(), any(), any());
1031  }
1032
1033  private long countMemStoreScanner(StoreScanner scanner) {
1034    if (scanner.currentScanners == null) {
1035      return 0;
1036    }
1037    return scanner.currentScanners.stream().filter(s -> !s.isFileScanner()).count();
1038  }
1039
1040  @Test
1041  public void testNumberOfMemStoreScannersAfterFlush() throws IOException {
1042    long seqId = 100;
1043    long timestamp = EnvironmentEdgeManager.currentTime();
1044    Cell cell0 = CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(row).setFamily(family)
1045      .setQualifier(qf1).setTimestamp(timestamp).setType(Cell.Type.Put).setValue(qf1).build();
1046    PrivateCellUtil.setSequenceId(cell0, seqId);
1047    testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Collections.emptyList());
1048
1049    Cell cell1 = CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(row).setFamily(family)
1050      .setQualifier(qf2).setTimestamp(timestamp).setType(Cell.Type.Put).setValue(qf1).build();
1051    PrivateCellUtil.setSequenceId(cell1, seqId);
1052    testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Arrays.asList(cell1));
1053
1054    seqId = 101;
1055    timestamp = EnvironmentEdgeManager.currentTime();
1056    Cell cell2 = CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(row2).setFamily(family)
1057      .setQualifier(qf2).setTimestamp(timestamp).setType(Cell.Type.Put).setValue(qf1).build();
1058    PrivateCellUtil.setSequenceId(cell2, seqId);
1059    testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Arrays.asList(cell1, cell2));
1060  }
1061
1062  private void testNumberOfMemStoreScannersAfterFlush(List<Cell> inputCellsBeforeSnapshot,
1063    List<Cell> inputCellsAfterSnapshot) throws IOException {
1064    init(this.name.getMethodName() + "-" + inputCellsBeforeSnapshot.size());
1065    TreeSet<byte[]> quals = new TreeSet<>(Bytes.BYTES_COMPARATOR);
1066    long seqId = Long.MIN_VALUE;
1067    for (Cell c : inputCellsBeforeSnapshot) {
1068      quals.add(CellUtil.cloneQualifier(c));
1069      seqId = Math.max(seqId, c.getSequenceId());
1070    }
1071    for (Cell c : inputCellsAfterSnapshot) {
1072      quals.add(CellUtil.cloneQualifier(c));
1073      seqId = Math.max(seqId, c.getSequenceId());
1074    }
1075    inputCellsBeforeSnapshot.forEach(c -> store.add(c, null));
1076    StoreFlushContext storeFlushCtx = store.createFlushContext(id++, FlushLifeCycleTracker.DUMMY);
1077    storeFlushCtx.prepare();
1078    inputCellsAfterSnapshot.forEach(c -> store.add(c, null));
1079    int numberOfMemScannersBeforeFlush = inputCellsAfterSnapshot.isEmpty() ? 1 : 2;
1080    try (StoreScanner s = (StoreScanner) store.getScanner(new Scan(), quals, seqId)) {
1081      // snapshot + active (if inputCellsAfterSnapshot isn't empty)
1082      assertEquals(numberOfMemScannersBeforeFlush, countMemStoreScanner(s));
1083      storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));
1084      storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));
1085      // snapshot has no data after flush
1086      int numberOfMemScannersAfterFlush = inputCellsAfterSnapshot.isEmpty() ? 0 : 1;
1087      boolean more;
1088      int cellCount = 0;
1089      do {
1090        List<Cell> cells = new ArrayList<>();
1091        more = s.next(cells);
1092        cellCount += cells.size();
1093        assertEquals(more ? numberOfMemScannersAfterFlush : 0, countMemStoreScanner(s));
1094      } while (more);
1095      assertEquals(
1096        "The number of cells added before snapshot is " + inputCellsBeforeSnapshot.size()
1097          + ", The number of cells added after snapshot is " + inputCellsAfterSnapshot.size(),
1098        inputCellsBeforeSnapshot.size() + inputCellsAfterSnapshot.size(), cellCount);
1099      // the current scanners is cleared
1100      assertEquals(0, countMemStoreScanner(s));
1101    }
1102  }
1103
1104  private Cell createCell(byte[] qualifier, long ts, long sequenceId, byte[] value)
1105    throws IOException {
1106    return createCell(row, qualifier, ts, sequenceId, value);
1107  }
1108
1109  private Cell createCell(byte[] row, byte[] qualifier, long ts, long sequenceId, byte[] value)
1110    throws IOException {
1111    Cell c = CellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(row).setFamily(family)
1112      .setQualifier(qualifier).setTimestamp(ts).setType(Cell.Type.Put).setValue(value).build();
1113    PrivateCellUtil.setSequenceId(c, sequenceId);
1114    return c;
1115  }
1116
1117  @Test
1118  public void testFlushBeforeCompletingScanWoFilter() throws IOException, InterruptedException {
1119    final AtomicBoolean timeToGoNextRow = new AtomicBoolean(false);
1120    final int expectedSize = 3;
1121    testFlushBeforeCompletingScan(new MyListHook() {
1122      @Override
1123      public void hook(int currentSize) {
1124        if (currentSize == expectedSize - 1) {
1125          try {
1126            flushStore(store, id++);
1127            timeToGoNextRow.set(true);
1128          } catch (IOException e) {
1129            throw new RuntimeException(e);
1130          }
1131        }
1132      }
1133    }, new FilterBase() {
1134      @Override
1135      public Filter.ReturnCode filterCell(final Cell c) throws IOException {
1136        return ReturnCode.INCLUDE;
1137      }
1138    }, expectedSize);
1139  }
1140
1141  @Test
1142  public void testFlushBeforeCompletingScanWithFilter() throws IOException, InterruptedException {
1143    final AtomicBoolean timeToGoNextRow = new AtomicBoolean(false);
1144    final int expectedSize = 2;
1145    testFlushBeforeCompletingScan(new MyListHook() {
1146      @Override
1147      public void hook(int currentSize) {
1148        if (currentSize == expectedSize - 1) {
1149          try {
1150            flushStore(store, id++);
1151            timeToGoNextRow.set(true);
1152          } catch (IOException e) {
1153            throw new RuntimeException(e);
1154          }
1155        }
1156      }
1157    }, new FilterBase() {
1158      @Override
1159      public Filter.ReturnCode filterCell(final Cell c) throws IOException {
1160        if (timeToGoNextRow.get()) {
1161          timeToGoNextRow.set(false);
1162          return ReturnCode.NEXT_ROW;
1163        } else {
1164          return ReturnCode.INCLUDE;
1165        }
1166      }
1167    }, expectedSize);
1168  }
1169
1170  @Test
1171  public void testFlushBeforeCompletingScanWithFilterHint()
1172    throws IOException, InterruptedException {
1173    final AtomicBoolean timeToGetHint = new AtomicBoolean(false);
1174    final int expectedSize = 2;
1175    testFlushBeforeCompletingScan(new MyListHook() {
1176      @Override
1177      public void hook(int currentSize) {
1178        if (currentSize == expectedSize - 1) {
1179          try {
1180            flushStore(store, id++);
1181            timeToGetHint.set(true);
1182          } catch (IOException e) {
1183            throw new RuntimeException(e);
1184          }
1185        }
1186      }
1187    }, new FilterBase() {
1188      @Override
1189      public Filter.ReturnCode filterCell(final Cell c) throws IOException {
1190        if (timeToGetHint.get()) {
1191          timeToGetHint.set(false);
1192          return Filter.ReturnCode.SEEK_NEXT_USING_HINT;
1193        } else {
1194          return Filter.ReturnCode.INCLUDE;
1195        }
1196      }
1197
1198      @Override
1199      public Cell getNextCellHint(Cell currentCell) throws IOException {
1200        return currentCell;
1201      }
1202    }, expectedSize);
1203  }
1204
1205  private void testFlushBeforeCompletingScan(MyListHook hook, Filter filter, int expectedSize)
1206    throws IOException, InterruptedException {
1207    Configuration conf = HBaseConfiguration.create();
1208    byte[] r0 = Bytes.toBytes("row0");
1209    byte[] r1 = Bytes.toBytes("row1");
1210    byte[] r2 = Bytes.toBytes("row2");
1211    byte[] value0 = Bytes.toBytes("value0");
1212    byte[] value1 = Bytes.toBytes("value1");
1213    byte[] value2 = Bytes.toBytes("value2");
1214    MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();
1215    long ts = EnvironmentEdgeManager.currentTime();
1216    long seqId = 100;
1217    init(name.getMethodName(), conf, TableDescriptorBuilder.newBuilder(TableName.valueOf(table)),
1218      ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(1).build(),
1219      new MyStoreHook() {
1220        @Override
1221        public long getSmallestReadPoint(HStore store) {
1222          return seqId + 3;
1223        }
1224      });
1225    // The cells having the value0 won't be flushed to disk because the value of max version is 1
1226    store.add(createCell(r0, qf1, ts, seqId, value0), memStoreSizing);
1227    store.add(createCell(r0, qf2, ts, seqId, value0), memStoreSizing);
1228    store.add(createCell(r0, qf3, ts, seqId, value0), memStoreSizing);
1229    store.add(createCell(r1, qf1, ts + 1, seqId + 1, value1), memStoreSizing);
1230    store.add(createCell(r1, qf2, ts + 1, seqId + 1, value1), memStoreSizing);
1231    store.add(createCell(r1, qf3, ts + 1, seqId + 1, value1), memStoreSizing);
1232    store.add(createCell(r2, qf1, ts + 2, seqId + 2, value2), memStoreSizing);
1233    store.add(createCell(r2, qf2, ts + 2, seqId + 2, value2), memStoreSizing);
1234    store.add(createCell(r2, qf3, ts + 2, seqId + 2, value2), memStoreSizing);
1235    store.add(createCell(r1, qf1, ts + 3, seqId + 3, value1), memStoreSizing);
1236    store.add(createCell(r1, qf2, ts + 3, seqId + 3, value1), memStoreSizing);
1237    store.add(createCell(r1, qf3, ts + 3, seqId + 3, value1), memStoreSizing);
1238    List<Cell> myList = new MyList<>(hook);
1239    Scan scan = new Scan().withStartRow(r1).setFilter(filter);
1240    try (InternalScanner scanner = (InternalScanner) store.getScanner(scan, null, seqId + 3)) {
1241      // r1
1242      scanner.next(myList);
1243      assertEquals(expectedSize, myList.size());
1244      for (Cell c : myList) {
1245        byte[] actualValue = CellUtil.cloneValue(c);
1246        assertTrue("expected:" + Bytes.toStringBinary(value1) + ", actual:"
1247          + Bytes.toStringBinary(actualValue), Bytes.equals(actualValue, value1));
1248      }
1249      List<Cell> normalList = new ArrayList<>(3);
1250      // r2
1251      scanner.next(normalList);
1252      assertEquals(3, normalList.size());
1253      for (Cell c : normalList) {
1254        byte[] actualValue = CellUtil.cloneValue(c);
1255        assertTrue("expected:" + Bytes.toStringBinary(value2) + ", actual:"
1256          + Bytes.toStringBinary(actualValue), Bytes.equals(actualValue, value2));
1257      }
1258    }
1259  }
1260
1261  @Test
1262  public void testCreateScannerAndSnapshotConcurrently() throws IOException, InterruptedException {
1263    Configuration conf = HBaseConfiguration.create();
1264    conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStore.class.getName());
1265    init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)
1266      .setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());
1267    byte[] value = Bytes.toBytes("value");
1268    MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();
1269    long ts = EnvironmentEdgeManager.currentTime();
1270    long seqId = 100;
1271    // older data whihc shouldn't be "seen" by client
1272    store.add(createCell(qf1, ts, seqId, value), memStoreSizing);
1273    store.add(createCell(qf2, ts, seqId, value), memStoreSizing);
1274    store.add(createCell(qf3, ts, seqId, value), memStoreSizing);
1275    TreeSet<byte[]> quals = new TreeSet<>(Bytes.BYTES_COMPARATOR);
1276    quals.add(qf1);
1277    quals.add(qf2);
1278    quals.add(qf3);
1279    StoreFlushContext storeFlushCtx = store.createFlushContext(id++, FlushLifeCycleTracker.DUMMY);
1280    MyCompactingMemStore.START_TEST.set(true);
1281    Runnable flush = () -> {
1282      // this is blocked until we create first scanner from pipeline and snapshot -- phase (1/5)
1283      // recreate the active memstore -- phase (4/5)
1284      storeFlushCtx.prepare();
1285    };
1286    ExecutorService service = Executors.newSingleThreadExecutor();
1287    service.execute(flush);
1288    // we get scanner from pipeline and snapshot but they are empty. -- phase (2/5)
1289    // this is blocked until we recreate the active memstore -- phase (3/5)
1290    // we get scanner from active memstore but it is empty -- phase (5/5)
1291    InternalScanner scanner =
1292      (InternalScanner) store.getScanner(new Scan(new Get(row)), quals, seqId + 1);
1293    service.shutdown();
1294    service.awaitTermination(20, TimeUnit.SECONDS);
1295    try {
1296      try {
1297        List<Cell> results = new ArrayList<>();
1298        scanner.next(results);
1299        assertEquals(3, results.size());
1300        for (Cell c : results) {
1301          byte[] actualValue = CellUtil.cloneValue(c);
1302          assertTrue("expected:" + Bytes.toStringBinary(value) + ", actual:"
1303            + Bytes.toStringBinary(actualValue), Bytes.equals(actualValue, value));
1304        }
1305      } finally {
1306        scanner.close();
1307      }
1308    } finally {
1309      MyCompactingMemStore.START_TEST.set(false);
1310      storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));
1311      storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));
1312    }
1313  }
1314
1315  @Test
1316  public void testScanWithDoubleFlush() throws IOException {
1317    Configuration conf = HBaseConfiguration.create();
1318    // Initialize region
1319    MyStore myStore = initMyStore(name.getMethodName(), conf, new MyStoreHook() {
1320      @Override
1321      public void getScanners(MyStore store) throws IOException {
1322        final long tmpId = id++;
1323        ExecutorService s = Executors.newSingleThreadExecutor();
1324        s.execute(() -> {
1325          try {
1326            // flush the store before storescanner updates the scanners from store.
1327            // The current data will be flushed into files, and the memstore will
1328            // be clear.
1329            // -- phase (4/4)
1330            flushStore(store, tmpId);
1331          } catch (IOException ex) {
1332            throw new RuntimeException(ex);
1333          }
1334        });
1335        s.shutdown();
1336        try {
1337          // wait for the flush, the thread will be blocked in HStore#notifyChangedReadersObservers.
1338          s.awaitTermination(3, TimeUnit.SECONDS);
1339        } catch (InterruptedException ex) {
1340        }
1341      }
1342    });
1343    byte[] oldValue = Bytes.toBytes("oldValue");
1344    byte[] currentValue = Bytes.toBytes("currentValue");
1345    MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();
1346    long ts = EnvironmentEdgeManager.currentTime();
1347    long seqId = 100;
1348    // older data whihc shouldn't be "seen" by client
1349    myStore.add(createCell(qf1, ts, seqId, oldValue), memStoreSizing);
1350    myStore.add(createCell(qf2, ts, seqId, oldValue), memStoreSizing);
1351    myStore.add(createCell(qf3, ts, seqId, oldValue), memStoreSizing);
1352    long snapshotId = id++;
1353    // push older data into snapshot -- phase (1/4)
1354    StoreFlushContext storeFlushCtx =
1355      store.createFlushContext(snapshotId, FlushLifeCycleTracker.DUMMY);
1356    storeFlushCtx.prepare();
1357
1358    // insert current data into active -- phase (2/4)
1359    myStore.add(createCell(qf1, ts + 1, seqId + 1, currentValue), memStoreSizing);
1360    myStore.add(createCell(qf2, ts + 1, seqId + 1, currentValue), memStoreSizing);
1361    myStore.add(createCell(qf3, ts + 1, seqId + 1, currentValue), memStoreSizing);
1362    TreeSet<byte[]> quals = new TreeSet<>(Bytes.BYTES_COMPARATOR);
1363    quals.add(qf1);
1364    quals.add(qf2);
1365    quals.add(qf3);
1366    try (InternalScanner scanner =
1367      (InternalScanner) myStore.getScanner(new Scan(new Get(row)), quals, seqId + 1)) {
1368      // complete the flush -- phase (3/4)
1369      storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));
1370      storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));
1371
1372      List<Cell> results = new ArrayList<>();
1373      scanner.next(results);
1374      assertEquals(3, results.size());
1375      for (Cell c : results) {
1376        byte[] actualValue = CellUtil.cloneValue(c);
1377        assertTrue("expected:" + Bytes.toStringBinary(currentValue) + ", actual:"
1378          + Bytes.toStringBinary(actualValue), Bytes.equals(actualValue, currentValue));
1379      }
1380    }
1381  }
1382
1383  @Test
1384  public void testReclaimChunkWhenScaning() throws IOException {
1385    init("testReclaimChunkWhenScaning");
1386    long ts = EnvironmentEdgeManager.currentTime();
1387    long seqId = 100;
1388    byte[] value = Bytes.toBytes("value");
1389    // older data whihc shouldn't be "seen" by client
1390    store.add(createCell(qf1, ts, seqId, value), null);
1391    store.add(createCell(qf2, ts, seqId, value), null);
1392    store.add(createCell(qf3, ts, seqId, value), null);
1393    TreeSet<byte[]> quals = new TreeSet<>(Bytes.BYTES_COMPARATOR);
1394    quals.add(qf1);
1395    quals.add(qf2);
1396    quals.add(qf3);
1397    try (InternalScanner scanner =
1398      (InternalScanner) store.getScanner(new Scan(new Get(row)), quals, seqId)) {
1399      List<Cell> results = new MyList<>(size -> {
1400        switch (size) {
1401          // 1) we get the first cell (qf1)
1402          // 2) flush the data to have StoreScanner update inner scanners
1403          // 3) the chunk will be reclaimed after updaing
1404          case 1:
1405            try {
1406              flushStore(store, id++);
1407            } catch (IOException e) {
1408              throw new RuntimeException(e);
1409            }
1410            break;
1411          // 1) we get the second cell (qf2)
1412          // 2) add some cell to fill some byte into the chunk (we have only one chunk)
1413          case 2:
1414            try {
1415              byte[] newValue = Bytes.toBytes("newValue");
1416              // older data whihc shouldn't be "seen" by client
1417              store.add(createCell(qf1, ts + 1, seqId + 1, newValue), null);
1418              store.add(createCell(qf2, ts + 1, seqId + 1, newValue), null);
1419              store.add(createCell(qf3, ts + 1, seqId + 1, newValue), null);
1420            } catch (IOException e) {
1421              throw new RuntimeException(e);
1422            }
1423            break;
1424          default:
1425            break;
1426        }
1427      });
1428      scanner.next(results);
1429      assertEquals(3, results.size());
1430      for (Cell c : results) {
1431        byte[] actualValue = CellUtil.cloneValue(c);
1432        assertTrue("expected:" + Bytes.toStringBinary(value) + ", actual:"
1433          + Bytes.toStringBinary(actualValue), Bytes.equals(actualValue, value));
1434      }
1435    }
1436  }
1437
1438  /**
1439   * If there are two running InMemoryFlushRunnable, the later InMemoryFlushRunnable may change the
1440   * versionedList. And the first InMemoryFlushRunnable will use the chagned versionedList to remove
1441   * the corresponding segments. In short, there will be some segements which isn't in merge are
1442   * removed.
1443   */
1444  @Test
1445  public void testRunDoubleMemStoreCompactors() throws IOException, InterruptedException {
1446    int flushSize = 500;
1447    Configuration conf = HBaseConfiguration.create();
1448    conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStoreWithCustomCompactor.class.getName());
1449    conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.25);
1450    MyCompactingMemStoreWithCustomCompactor.RUNNER_COUNT.set(0);
1451    conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(flushSize));
1452    // Set the lower threshold to invoke the "MERGE" policy
1453    conf.set(MemStoreCompactionStrategy.COMPACTING_MEMSTORE_THRESHOLD_KEY, String.valueOf(0));
1454    init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)
1455      .setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());
1456    byte[] value = Bytes.toBytes("thisisavarylargevalue");
1457    MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();
1458    long ts = EnvironmentEdgeManager.currentTime();
1459    long seqId = 100;
1460    // older data whihc shouldn't be "seen" by client
1461    store.add(createCell(qf1, ts, seqId, value), memStoreSizing);
1462    store.add(createCell(qf2, ts, seqId, value), memStoreSizing);
1463    store.add(createCell(qf3, ts, seqId, value), memStoreSizing);
1464    assertEquals(1, MyCompactingMemStoreWithCustomCompactor.RUNNER_COUNT.get());
1465    StoreFlushContext storeFlushCtx = store.createFlushContext(id++, FlushLifeCycleTracker.DUMMY);
1466    storeFlushCtx.prepare();
1467    // This shouldn't invoke another in-memory flush because the first compactor thread
1468    // hasn't accomplished the in-memory compaction.
1469    store.add(createCell(qf1, ts + 1, seqId + 1, value), memStoreSizing);
1470    store.add(createCell(qf1, ts + 1, seqId + 1, value), memStoreSizing);
1471    store.add(createCell(qf1, ts + 1, seqId + 1, value), memStoreSizing);
1472    assertEquals(1, MyCompactingMemStoreWithCustomCompactor.RUNNER_COUNT.get());
1473    // okay. Let the compaction be completed
1474    MyMemStoreCompactor.START_COMPACTOR_LATCH.countDown();
1475    CompactingMemStore mem = (CompactingMemStore) ((HStore) store).memstore;
1476    while (mem.isMemStoreFlushingInMemory()) {
1477      TimeUnit.SECONDS.sleep(1);
1478    }
1479    // This should invoke another in-memory flush.
1480    store.add(createCell(qf1, ts + 2, seqId + 2, value), memStoreSizing);
1481    store.add(createCell(qf1, ts + 2, seqId + 2, value), memStoreSizing);
1482    store.add(createCell(qf1, ts + 2, seqId + 2, value), memStoreSizing);
1483    assertEquals(2, MyCompactingMemStoreWithCustomCompactor.RUNNER_COUNT.get());
1484    conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
1485      String.valueOf(TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE));
1486    storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));
1487    storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));
1488  }
1489
1490  @Test
1491  public void testAge() throws IOException {
1492    long currentTime = EnvironmentEdgeManager.currentTime();
1493    ManualEnvironmentEdge edge = new ManualEnvironmentEdge();
1494    edge.setValue(currentTime);
1495    EnvironmentEdgeManager.injectEdge(edge);
1496    Configuration conf = TEST_UTIL.getConfiguration();
1497    ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.of(family);
1498    initHRegion(name.getMethodName(), conf,
1499      TableDescriptorBuilder.newBuilder(TableName.valueOf(table)), hcd, null, false);
1500    HStore store = new HStore(region, hcd, conf, false) {
1501
1502      @Override
1503      protected StoreEngine<?, ?, ?, ?> createStoreEngine(HStore store, Configuration conf,
1504        CellComparator kvComparator) throws IOException {
1505        List<HStoreFile> storefiles =
1506          Arrays.asList(mockStoreFile(currentTime - 10), mockStoreFile(currentTime - 100),
1507            mockStoreFile(currentTime - 1000), mockStoreFile(currentTime - 10000));
1508        StoreFileManager sfm = mock(StoreFileManager.class);
1509        when(sfm.getStorefiles()).thenReturn(storefiles);
1510        StoreEngine<?, ?, ?, ?> storeEngine = mock(StoreEngine.class);
1511        when(storeEngine.getStoreFileManager()).thenReturn(sfm);
1512        return storeEngine;
1513      }
1514    };
1515    assertEquals(10L, store.getMinStoreFileAge().getAsLong());
1516    assertEquals(10000L, store.getMaxStoreFileAge().getAsLong());
1517    assertEquals((10 + 100 + 1000 + 10000) / 4.0, store.getAvgStoreFileAge().getAsDouble(), 1E-4);
1518  }
1519
1520  private HStoreFile mockStoreFile(long createdTime) {
1521    StoreFileInfo info = mock(StoreFileInfo.class);
1522    when(info.getCreatedTimestamp()).thenReturn(createdTime);
1523    HStoreFile sf = mock(HStoreFile.class);
1524    when(sf.getReader()).thenReturn(mock(StoreFileReader.class));
1525    when(sf.isHFile()).thenReturn(true);
1526    when(sf.getFileInfo()).thenReturn(info);
1527    return sf;
1528  }
1529
1530  private MyStore initMyStore(String methodName, Configuration conf, MyStoreHook hook)
1531    throws IOException {
1532    return (MyStore) init(methodName, conf,
1533      TableDescriptorBuilder.newBuilder(TableName.valueOf(table)),
1534      ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(5).build(), hook);
1535  }
1536
1537  private static class MyStore extends HStore {
1538    private final MyStoreHook hook;
1539
1540    MyStore(final HRegion region, final ColumnFamilyDescriptor family,
1541      final Configuration confParam, MyStoreHook hook, boolean switchToPread) throws IOException {
1542      super(region, family, confParam, false);
1543      this.hook = hook;
1544    }
1545
1546    @Override
1547    public List<KeyValueScanner> getScanners(List<HStoreFile> files, boolean cacheBlocks,
1548      boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow,
1549      boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt,
1550      boolean includeMemstoreScanner) throws IOException {
1551      hook.getScanners(this);
1552      return super.getScanners(files, cacheBlocks, usePread, isCompaction, matcher, startRow, true,
1553        stopRow, false, readPt, includeMemstoreScanner);
1554    }
1555
1556    @Override
1557    public long getSmallestReadPoint() {
1558      return hook.getSmallestReadPoint(this);
1559    }
1560  }
1561
1562  private abstract static class MyStoreHook {
1563
1564    void getScanners(MyStore store) throws IOException {
1565    }
1566
1567    long getSmallestReadPoint(HStore store) {
1568      return store.getHRegion().getSmallestReadPoint();
1569    }
1570  }
1571
1572  @Test
1573  public void testSwitchingPreadtoStreamParallelyWithCompactionDischarger() throws Exception {
1574    Configuration conf = HBaseConfiguration.create();
1575    conf.set("hbase.hstore.engine.class", DummyStoreEngine.class.getName());
1576    conf.setLong(StoreScanner.STORESCANNER_PREAD_MAX_BYTES, 0);
1577    // Set the lower threshold to invoke the "MERGE" policy
1578    MyStore store = initMyStore(name.getMethodName(), conf, new MyStoreHook() {
1579    });
1580    MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();
1581    long ts = EnvironmentEdgeManager.currentTime();
1582    long seqID = 1L;
1583    // Add some data to the region and do some flushes
1584    for (int i = 1; i < 10; i++) {
1585      store.add(createCell(Bytes.toBytes("row" + i), qf1, ts, seqID++, Bytes.toBytes("")),
1586        memStoreSizing);
1587    }
1588    // flush them
1589    flushStore(store, seqID);
1590    for (int i = 11; i < 20; i++) {
1591      store.add(createCell(Bytes.toBytes("row" + i), qf1, ts, seqID++, Bytes.toBytes("")),
1592        memStoreSizing);
1593    }
1594    // flush them
1595    flushStore(store, seqID);
1596    for (int i = 21; i < 30; i++) {
1597      store.add(createCell(Bytes.toBytes("row" + i), qf1, ts, seqID++, Bytes.toBytes("")),
1598        memStoreSizing);
1599    }
1600    // flush them
1601    flushStore(store, seqID);
1602
1603    assertEquals(3, store.getStorefilesCount());
1604    Scan scan = new Scan();
1605    scan.addFamily(family);
1606    Collection<HStoreFile> storefiles2 = store.getStorefiles();
1607    ArrayList<HStoreFile> actualStorefiles = Lists.newArrayList(storefiles2);
1608    StoreScanner storeScanner =
1609      (StoreScanner) store.getScanner(scan, scan.getFamilyMap().get(family), Long.MAX_VALUE);
1610    // get the current heap
1611    KeyValueHeap heap = storeScanner.heap;
1612    // create more store files
1613    for (int i = 31; i < 40; i++) {
1614      store.add(createCell(Bytes.toBytes("row" + i), qf1, ts, seqID++, Bytes.toBytes("")),
1615        memStoreSizing);
1616    }
1617    // flush them
1618    flushStore(store, seqID);
1619
1620    for (int i = 41; i < 50; i++) {
1621      store.add(createCell(Bytes.toBytes("row" + i), qf1, ts, seqID++, Bytes.toBytes("")),
1622        memStoreSizing);
1623    }
1624    // flush them
1625    flushStore(store, seqID);
1626    storefiles2 = store.getStorefiles();
1627    ArrayList<HStoreFile> actualStorefiles1 = Lists.newArrayList(storefiles2);
1628    actualStorefiles1.removeAll(actualStorefiles);
1629    // Do compaction
1630    MyThread thread = new MyThread(storeScanner);
1631    thread.start();
1632    store.replaceStoreFiles(actualStorefiles, actualStorefiles1, false);
1633    thread.join();
1634    KeyValueHeap heap2 = thread.getHeap();
1635    assertFalse(heap.equals(heap2));
1636  }
1637
1638  @Test
1639  public void testMaxPreadBytesConfiguredToBeLessThanZero() throws Exception {
1640    Configuration conf = HBaseConfiguration.create();
1641    conf.set("hbase.hstore.engine.class", DummyStoreEngine.class.getName());
1642    // Set 'hbase.storescanner.pread.max.bytes' < 0, so that StoreScanner will be a STREAM type.
1643    conf.setLong(StoreScanner.STORESCANNER_PREAD_MAX_BYTES, -1);
1644    MyStore store = initMyStore(name.getMethodName(), conf, new MyStoreHook() {
1645    });
1646    Scan scan = new Scan();
1647    scan.addFamily(family);
1648    // ReadType on Scan is still DEFAULT only.
1649    assertEquals(ReadType.DEFAULT, scan.getReadType());
1650    StoreScanner storeScanner =
1651      (StoreScanner) store.getScanner(scan, scan.getFamilyMap().get(family), Long.MAX_VALUE);
1652    assertFalse(storeScanner.isScanUsePread());
1653  }
1654
1655  @Test
1656  public void testSpaceQuotaChangeAfterReplacement() throws IOException {
1657    final TableName tn = TableName.valueOf(name.getMethodName());
1658    init(name.getMethodName());
1659
1660    RegionSizeStoreImpl sizeStore = new RegionSizeStoreImpl();
1661
1662    HStoreFile sf1 = mockStoreFileWithLength(1024L);
1663    HStoreFile sf2 = mockStoreFileWithLength(2048L);
1664    HStoreFile sf3 = mockStoreFileWithLength(4096L);
1665    HStoreFile sf4 = mockStoreFileWithLength(8192L);
1666
1667    RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tn).setStartKey(Bytes.toBytes("a"))
1668      .setEndKey(Bytes.toBytes("b")).build();
1669
1670    // Compacting two files down to one, reducing size
1671    sizeStore.put(regionInfo, 1024L + 4096L);
1672    store.updateSpaceQuotaAfterFileReplacement(sizeStore, regionInfo, Arrays.asList(sf1, sf3),
1673      Arrays.asList(sf2));
1674
1675    assertEquals(2048L, sizeStore.getRegionSize(regionInfo).getSize());
1676
1677    // The same file length in and out should have no change
1678    store.updateSpaceQuotaAfterFileReplacement(sizeStore, regionInfo, Arrays.asList(sf2),
1679      Arrays.asList(sf2));
1680
1681    assertEquals(2048L, sizeStore.getRegionSize(regionInfo).getSize());
1682
1683    // Increase the total size used
1684    store.updateSpaceQuotaAfterFileReplacement(sizeStore, regionInfo, Arrays.asList(sf2),
1685      Arrays.asList(sf3));
1686
1687    assertEquals(4096L, sizeStore.getRegionSize(regionInfo).getSize());
1688
1689    RegionInfo regionInfo2 = RegionInfoBuilder.newBuilder(tn).setStartKey(Bytes.toBytes("b"))
1690      .setEndKey(Bytes.toBytes("c")).build();
1691    store.updateSpaceQuotaAfterFileReplacement(sizeStore, regionInfo2, null, Arrays.asList(sf4));
1692
1693    assertEquals(8192L, sizeStore.getRegionSize(regionInfo2).getSize());
1694  }
1695
1696  @Test
1697  public void testHFileContextSetWithCFAndTable() throws Exception {
1698    init(this.name.getMethodName());
1699    StoreFileWriter writer = store.getStoreEngine()
1700      .createWriter(CreateStoreFileWriterParams.create().maxKeyCount(10000L)
1701        .compression(Compression.Algorithm.NONE).isCompaction(true).includeMVCCReadpoint(true)
1702        .includesTag(false).shouldDropBehind(true));
1703    HFileContext hFileContext = writer.getHFileWriter().getFileContext();
1704    assertArrayEquals(family, hFileContext.getColumnFamily());
1705    assertArrayEquals(table, hFileContext.getTableName());
1706  }
1707
1708  // This test is for HBASE-26026, HBase Write be stuck when active segment has no cell
1709  // but its dataSize exceeds inmemoryFlushSize
1710  @Test
1711  public void testCompactingMemStoreNoCellButDataSizeExceedsInmemoryFlushSize()
1712    throws IOException, InterruptedException {
1713    Configuration conf = HBaseConfiguration.create();
1714
1715    byte[] smallValue = new byte[3];
1716    byte[] largeValue = new byte[9];
1717    final long timestamp = EnvironmentEdgeManager.currentTime();
1718    final long seqId = 100;
1719    final Cell smallCell = createCell(qf1, timestamp, seqId, smallValue);
1720    final Cell largeCell = createCell(qf2, timestamp, seqId, largeValue);
1721    int smallCellByteSize = MutableSegment.getCellLength(smallCell);
1722    int largeCellByteSize = MutableSegment.getCellLength(largeCell);
1723    int flushByteSize = smallCellByteSize + largeCellByteSize - 2;
1724
1725    // set CompactingMemStore.inmemoryFlushSize to flushByteSize.
1726    conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStore2.class.getName());
1727    conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.005);
1728    conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(flushByteSize * 200));
1729
1730    init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)
1731      .setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());
1732
1733    MyCompactingMemStore2 myCompactingMemStore = ((MyCompactingMemStore2) store.memstore);
1734    assertTrue((int) (myCompactingMemStore.getInmemoryFlushSize()) == flushByteSize);
1735    myCompactingMemStore.smallCellPreUpdateCounter.set(0);
1736    myCompactingMemStore.largeCellPreUpdateCounter.set(0);
1737
1738    final AtomicReference<Throwable> exceptionRef = new AtomicReference<Throwable>();
1739    Thread smallCellThread = new Thread(() -> {
1740      try {
1741        store.add(smallCell, new NonThreadSafeMemStoreSizing());
1742      } catch (Throwable exception) {
1743        exceptionRef.set(exception);
1744      }
1745    });
1746    smallCellThread.setName(MyCompactingMemStore2.SMALL_CELL_THREAD_NAME);
1747    smallCellThread.start();
1748
1749    String oldThreadName = Thread.currentThread().getName();
1750    try {
1751      /**
1752       * 1.smallCellThread enters CompactingMemStore.checkAndAddToActiveSize first, then
1753       * largeCellThread enters CompactingMemStore.checkAndAddToActiveSize, and then largeCellThread
1754       * invokes flushInMemory.
1755       * <p/>
1756       * 2. After largeCellThread finished CompactingMemStore.flushInMemory method, smallCellThread
1757       * can add cell to currentActive . That is to say when largeCellThread called flushInMemory
1758       * method, CompactingMemStore.active has no cell.
1759       */
1760      Thread.currentThread().setName(MyCompactingMemStore2.LARGE_CELL_THREAD_NAME);
1761      store.add(largeCell, new NonThreadSafeMemStoreSizing());
1762      smallCellThread.join();
1763
1764      for (int i = 0; i < 100; i++) {
1765        long currentTimestamp = timestamp + 100 + i;
1766        Cell cell = createCell(qf2, currentTimestamp, seqId, largeValue);
1767        store.add(cell, new NonThreadSafeMemStoreSizing());
1768      }
1769    } finally {
1770      Thread.currentThread().setName(oldThreadName);
1771    }
1772
1773    assertTrue(exceptionRef.get() == null);
1774
1775  }
1776
1777  // This test is for HBASE-26210, HBase Write be stuck when there is cell which size exceeds
1778  // InmemoryFlushSize
1779  @Test(timeout = 60000)
1780  public void testCompactingMemStoreCellExceedInmemoryFlushSize() throws Exception {
1781    Configuration conf = HBaseConfiguration.create();
1782    conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStore6.class.getName());
1783
1784    init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)
1785      .setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());
1786
1787    MyCompactingMemStore6 myCompactingMemStore = ((MyCompactingMemStore6) store.memstore);
1788
1789    int size = (int) (myCompactingMemStore.getInmemoryFlushSize());
1790    byte[] value = new byte[size + 1];
1791
1792    MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();
1793    long timestamp = EnvironmentEdgeManager.currentTime();
1794    long seqId = 100;
1795    Cell cell = createCell(qf1, timestamp, seqId, value);
1796    int cellByteSize = MutableSegment.getCellLength(cell);
1797    store.add(cell, memStoreSizing);
1798    assertTrue(memStoreSizing.getCellsCount() == 1);
1799    assertTrue(memStoreSizing.getDataSize() == cellByteSize);
1800    // Waiting the in memory compaction completed, see HBASE-26438
1801    myCompactingMemStore.inMemoryCompactionEndCyclicBarrier.await();
1802  }
1803
1804  // This test is for HBASE-26210 also, test write large cell and small cell concurrently when
1805  // InmemoryFlushSize is smaller,equal with and larger than cell size.
1806  @Test
1807  public void testCompactingMemStoreWriteLargeCellAndSmallCellConcurrently()
1808    throws IOException, InterruptedException {
1809    doWriteTestLargeCellAndSmallCellConcurrently(
1810      (smallCellByteSize, largeCellByteSize) -> largeCellByteSize - 1);
1811    doWriteTestLargeCellAndSmallCellConcurrently(
1812      (smallCellByteSize, largeCellByteSize) -> largeCellByteSize);
1813    doWriteTestLargeCellAndSmallCellConcurrently(
1814      (smallCellByteSize, largeCellByteSize) -> smallCellByteSize + largeCellByteSize - 1);
1815    doWriteTestLargeCellAndSmallCellConcurrently(
1816      (smallCellByteSize, largeCellByteSize) -> smallCellByteSize + largeCellByteSize);
1817    doWriteTestLargeCellAndSmallCellConcurrently(
1818      (smallCellByteSize, largeCellByteSize) -> smallCellByteSize + largeCellByteSize + 1);
1819  }
1820
1821  private void doWriteTestLargeCellAndSmallCellConcurrently(IntBinaryOperator getFlushByteSize)
1822    throws IOException, InterruptedException {
1823
1824    Configuration conf = HBaseConfiguration.create();
1825
1826    byte[] smallValue = new byte[3];
1827    byte[] largeValue = new byte[100];
1828    final long timestamp = EnvironmentEdgeManager.currentTime();
1829    final long seqId = 100;
1830    final Cell smallCell = createCell(qf1, timestamp, seqId, smallValue);
1831    final Cell largeCell = createCell(qf2, timestamp, seqId, largeValue);
1832    int smallCellByteSize = MutableSegment.getCellLength(smallCell);
1833    int largeCellByteSize = MutableSegment.getCellLength(largeCell);
1834    int flushByteSize = getFlushByteSize.applyAsInt(smallCellByteSize, largeCellByteSize);
1835    boolean flushByteSizeLessThanSmallAndLargeCellSize =
1836      flushByteSize < (smallCellByteSize + largeCellByteSize);
1837
1838    conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStore3.class.getName());
1839    conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.005);
1840    conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(flushByteSize * 200));
1841
1842    init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)
1843      .setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());
1844
1845    MyCompactingMemStore3 myCompactingMemStore = ((MyCompactingMemStore3) store.memstore);
1846    assertTrue((int) (myCompactingMemStore.getInmemoryFlushSize()) == flushByteSize);
1847    myCompactingMemStore.disableCompaction();
1848    if (flushByteSizeLessThanSmallAndLargeCellSize) {
1849      myCompactingMemStore.flushByteSizeLessThanSmallAndLargeCellSize = true;
1850    } else {
1851      myCompactingMemStore.flushByteSizeLessThanSmallAndLargeCellSize = false;
1852    }
1853
1854    final ThreadSafeMemStoreSizing memStoreSizing = new ThreadSafeMemStoreSizing();
1855    final AtomicLong totalCellByteSize = new AtomicLong(0);
1856    final AtomicReference<Throwable> exceptionRef = new AtomicReference<Throwable>();
1857    Thread smallCellThread = new Thread(() -> {
1858      try {
1859        for (int i = 1; i <= MyCompactingMemStore3.CELL_COUNT; i++) {
1860          long currentTimestamp = timestamp + i;
1861          Cell cell = createCell(qf1, currentTimestamp, seqId, smallValue);
1862          totalCellByteSize.addAndGet(MutableSegment.getCellLength(cell));
1863          store.add(cell, memStoreSizing);
1864        }
1865      } catch (Throwable exception) {
1866        exceptionRef.set(exception);
1867
1868      }
1869    });
1870    smallCellThread.setName(MyCompactingMemStore3.SMALL_CELL_THREAD_NAME);
1871    smallCellThread.start();
1872
1873    String oldThreadName = Thread.currentThread().getName();
1874    try {
1875      /**
1876       * When flushByteSizeLessThanSmallAndLargeCellSize is true:
1877       * </p>
1878       * 1.smallCellThread enters MyCompactingMemStore3.checkAndAddToActiveSize first, then
1879       * largeCellThread enters MyCompactingMemStore3.checkAndAddToActiveSize, and then
1880       * largeCellThread invokes flushInMemory.
1881       * <p/>
1882       * 2. After largeCellThread finished CompactingMemStore.flushInMemory method, smallCellThread
1883       * can run into MyCompactingMemStore3.checkAndAddToActiveSize again.
1884       * <p/>
1885       * When flushByteSizeLessThanSmallAndLargeCellSize is false: smallCellThread and
1886       * largeCellThread concurrently write one cell and wait each other, and then write another
1887       * cell etc.
1888       */
1889      Thread.currentThread().setName(MyCompactingMemStore3.LARGE_CELL_THREAD_NAME);
1890      for (int i = 1; i <= MyCompactingMemStore3.CELL_COUNT; i++) {
1891        long currentTimestamp = timestamp + i;
1892        Cell cell = createCell(qf2, currentTimestamp, seqId, largeValue);
1893        totalCellByteSize.addAndGet(MutableSegment.getCellLength(cell));
1894        store.add(cell, memStoreSizing);
1895      }
1896      smallCellThread.join();
1897
1898      assertTrue(exceptionRef.get() == null);
1899      assertTrue(memStoreSizing.getCellsCount() == (MyCompactingMemStore3.CELL_COUNT * 2));
1900      assertTrue(memStoreSizing.getDataSize() == totalCellByteSize.get());
1901      if (flushByteSizeLessThanSmallAndLargeCellSize) {
1902        assertTrue(myCompactingMemStore.flushCounter.get() == MyCompactingMemStore3.CELL_COUNT);
1903      } else {
1904        assertTrue(
1905          myCompactingMemStore.flushCounter.get() <= (MyCompactingMemStore3.CELL_COUNT - 1));
1906      }
1907    } finally {
1908      Thread.currentThread().setName(oldThreadName);
1909    }
1910  }
1911
1912  /**
1913   * <pre>
1914   * This test is for HBASE-26384,
1915   * test {@link CompactingMemStore#flattenOneSegment} and {@link CompactingMemStore#snapshot()}
1916   * execute concurrently.
1917   * The threads sequence before HBASE-26384 is(The bug only exists for branch-2,and I add UTs
1918   * for both branch-2 and master):
1919   * 1. The {@link CompactingMemStore} size exceeds
1920   *    {@link CompactingMemStore#getInmemoryFlushSize()},the write thread adds a new
1921   *    {@link ImmutableSegment}  to the head of {@link CompactingMemStore#pipeline},and start a
1922   *    in memory compact thread to execute {@link CompactingMemStore#inMemoryCompaction}.
1923   * 2. The in memory compact thread starts and then stopping before
1924   *    {@link CompactingMemStore#flattenOneSegment}.
1925   * 3. The snapshot thread starts {@link CompactingMemStore#snapshot} concurrently,after the
1926   *    snapshot thread executing {@link CompactingMemStore#getImmutableSegments},the in memory
1927   *    compact thread continues.
1928   *    Assuming {@link VersionedSegmentsList#version} returned from
1929   *    {@link CompactingMemStore#getImmutableSegments} is v.
1930   * 4. The snapshot thread stopping before {@link CompactingMemStore#swapPipelineWithNull}.
1931   * 5. The in memory compact thread completes {@link CompactingMemStore#flattenOneSegment},
1932   *    {@link CompactionPipeline#version} is still v.
1933   * 6. The snapshot thread continues {@link CompactingMemStore#swapPipelineWithNull}, and because
1934   *    {@link CompactionPipeline#version} is v, {@link CompactingMemStore#swapPipelineWithNull}
1935   *    thinks it is successful and continue flushing,but the {@link ImmutableSegment} in
1936   *    {@link CompactionPipeline} has changed because
1937   *    {@link CompactingMemStore#flattenOneSegment},so the {@link ImmutableSegment} is not
1938   *    removed in fact and still remaining in {@link CompactionPipeline}.
1939   *
1940   * After HBASE-26384, the 5-6 step is changed to following, which is expected behavior:
1941   * 5. The in memory compact thread completes {@link CompactingMemStore#flattenOneSegment},
1942   *    {@link CompactingMemStore#flattenOneSegment} change {@link CompactionPipeline#version} to
1943   *    v+1.
1944   * 6. The snapshot thread continues {@link CompactingMemStore#swapPipelineWithNull}, and because
1945   *    {@link CompactionPipeline#version} is v+1, {@link CompactingMemStore#swapPipelineWithNull}
1946   *    failed and retry the while loop in {@link CompactingMemStore#pushPipelineToSnapshot} once
1947   *    again, because there is no concurrent {@link CompactingMemStore#inMemoryCompaction} now,
1948   *    {@link CompactingMemStore#swapPipelineWithNull} succeeds.
1949   * </pre>
1950   */
1951  @Test
1952  public void testFlattenAndSnapshotCompactingMemStoreConcurrently() throws Exception {
1953    Configuration conf = HBaseConfiguration.create();
1954
1955    byte[] smallValue = new byte[3];
1956    byte[] largeValue = new byte[9];
1957    final long timestamp = EnvironmentEdgeManager.currentTime();
1958    final long seqId = 100;
1959    final Cell smallCell = createCell(qf1, timestamp, seqId, smallValue);
1960    final Cell largeCell = createCell(qf2, timestamp, seqId, largeValue);
1961    int smallCellByteSize = MutableSegment.getCellLength(smallCell);
1962    int largeCellByteSize = MutableSegment.getCellLength(largeCell);
1963    int totalCellByteSize = (smallCellByteSize + largeCellByteSize);
1964    int flushByteSize = totalCellByteSize - 2;
1965
1966    // set CompactingMemStore.inmemoryFlushSize to flushByteSize.
1967    conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStore4.class.getName());
1968    conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.005);
1969    conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(flushByteSize * 200));
1970
1971    init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)
1972      .setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());
1973
1974    MyCompactingMemStore4 myCompactingMemStore = ((MyCompactingMemStore4) store.memstore);
1975    assertTrue((int) (myCompactingMemStore.getInmemoryFlushSize()) == flushByteSize);
1976
1977    store.add(smallCell, new NonThreadSafeMemStoreSizing());
1978    store.add(largeCell, new NonThreadSafeMemStoreSizing());
1979
1980    String oldThreadName = Thread.currentThread().getName();
1981    try {
1982      Thread.currentThread().setName(MyCompactingMemStore4.TAKE_SNAPSHOT_THREAD_NAME);
1983      /**
1984       * {@link CompactingMemStore#snapshot} must wait the in memory compact thread enters
1985       * {@link CompactingMemStore#flattenOneSegment},because {@link CompactingMemStore#snapshot}
1986       * would invoke {@link CompactingMemStore#stopCompaction}.
1987       */
1988      myCompactingMemStore.snapShotStartCyclicCyclicBarrier.await();
1989
1990      MemStoreSnapshot memStoreSnapshot = myCompactingMemStore.snapshot();
1991      myCompactingMemStore.inMemoryCompactionEndCyclicBarrier.await();
1992
1993      assertTrue(memStoreSnapshot.getCellsCount() == 2);
1994      assertTrue(((int) (memStoreSnapshot.getDataSize())) == totalCellByteSize);
1995      VersionedSegmentsList segments = myCompactingMemStore.getImmutableSegments();
1996      assertTrue(segments.getNumOfSegments() == 0);
1997      assertTrue(segments.getNumOfCells() == 0);
1998      assertTrue(myCompactingMemStore.setInMemoryCompactionFlagCounter.get() == 1);
1999      assertTrue(myCompactingMemStore.swapPipelineWithNullCounter.get() == 2);
2000    } finally {
2001      Thread.currentThread().setName(oldThreadName);
2002    }
2003  }
2004
2005  /**
2006   * <pre>
2007   * This test is for HBASE-26384,
2008   * test {@link CompactingMemStore#flattenOneSegment}{@link CompactingMemStore#snapshot()}
2009   * and writeMemStore execute concurrently.
2010   * The threads sequence before HBASE-26384 is(The bug only exists for branch-2,and I add UTs
2011   * for both branch-2 and master):
2012   * 1. The {@link CompactingMemStore} size exceeds
2013   *    {@link CompactingMemStore#getInmemoryFlushSize()},the write thread adds a new
2014   *    {@link ImmutableSegment}  to the head of {@link CompactingMemStore#pipeline},and start a
2015   *    in memory compact thread to execute {@link CompactingMemStore#inMemoryCompaction}.
2016   * 2. The in memory compact thread starts and then stopping before
2017   *    {@link CompactingMemStore#flattenOneSegment}.
2018   * 3. The snapshot thread starts {@link CompactingMemStore#snapshot} concurrently,after the
2019   *    snapshot thread executing {@link CompactingMemStore#getImmutableSegments},the in memory
2020   *    compact thread continues.
2021   *    Assuming {@link VersionedSegmentsList#version} returned from
2022   *    {@link CompactingMemStore#getImmutableSegments} is v.
2023   * 4. The snapshot thread stopping before {@link CompactingMemStore#swapPipelineWithNull}.
2024   * 5. The in memory compact thread completes {@link CompactingMemStore#flattenOneSegment},
2025   *    {@link CompactionPipeline#version} is still v.
2026   * 6. The snapshot thread continues {@link CompactingMemStore#swapPipelineWithNull}, and because
2027   *    {@link CompactionPipeline#version} is v, {@link CompactingMemStore#swapPipelineWithNull}
2028   *    thinks it is successful and continue flushing,but the {@link ImmutableSegment} in
2029   *    {@link CompactionPipeline} has changed because
2030   *    {@link CompactingMemStore#flattenOneSegment},so the {@link ImmutableSegment} is not
2031   *    removed in fact and still remaining in {@link CompactionPipeline}.
2032   *
2033   * After HBASE-26384, the 5-6 step is changed to following, which is expected behavior,
2034   * and I add step 7-8 to test there is new segment added before retry.
2035   * 5. The in memory compact thread completes {@link CompactingMemStore#flattenOneSegment},
2036   *    {@link CompactingMemStore#flattenOneSegment} change {@link CompactionPipeline#version} to
2037   *     v+1.
2038   * 6. The snapshot thread continues {@link CompactingMemStore#swapPipelineWithNull}, and because
2039   *    {@link CompactionPipeline#version} is v+1, {@link CompactingMemStore#swapPipelineWithNull}
2040   *    failed and retry,{@link VersionedSegmentsList#version} returned from
2041   *    {@link CompactingMemStore#getImmutableSegments} is v+1.
2042   * 7. The write thread continues writing to {@link CompactingMemStore} and
2043   *    {@link CompactingMemStore} size exceeds {@link CompactingMemStore#getInmemoryFlushSize()},
2044   *    {@link CompactingMemStore#flushInMemory(MutableSegment)} is called and a new
2045   *    {@link ImmutableSegment} is added to the head of {@link CompactingMemStore#pipeline},
2046   *    {@link CompactionPipeline#version} is still v+1.
2047   * 8. The snapshot thread continues {@link CompactingMemStore#swapPipelineWithNull}, and because
2048   *    {@link CompactionPipeline#version} is still v+1,
2049   *    {@link CompactingMemStore#swapPipelineWithNull} succeeds.The new {@link ImmutableSegment}
2050   *    remained at the head of {@link CompactingMemStore#pipeline},the old is removed by
2051   *    {@link CompactingMemStore#swapPipelineWithNull}.
2052   * </pre>
2053   */
2054  @Test
2055  public void testFlattenSnapshotWriteCompactingMemeStoreConcurrently() throws Exception {
2056    Configuration conf = HBaseConfiguration.create();
2057
2058    byte[] smallValue = new byte[3];
2059    byte[] largeValue = new byte[9];
2060    final long timestamp = EnvironmentEdgeManager.currentTime();
2061    final long seqId = 100;
2062    final Cell smallCell = createCell(qf1, timestamp, seqId, smallValue);
2063    final Cell largeCell = createCell(qf2, timestamp, seqId, largeValue);
2064    int smallCellByteSize = MutableSegment.getCellLength(smallCell);
2065    int largeCellByteSize = MutableSegment.getCellLength(largeCell);
2066    int firstWriteCellByteSize = (smallCellByteSize + largeCellByteSize);
2067    int flushByteSize = firstWriteCellByteSize - 2;
2068
2069    // set CompactingMemStore.inmemoryFlushSize to flushByteSize.
2070    conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStore5.class.getName());
2071    conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.005);
2072    conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(flushByteSize * 200));
2073
2074    init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)
2075      .setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());
2076
2077    final MyCompactingMemStore5 myCompactingMemStore = ((MyCompactingMemStore5) store.memstore);
2078    assertTrue((int) (myCompactingMemStore.getInmemoryFlushSize()) == flushByteSize);
2079
2080    store.add(smallCell, new NonThreadSafeMemStoreSizing());
2081    store.add(largeCell, new NonThreadSafeMemStoreSizing());
2082
2083    final AtomicReference<Throwable> exceptionRef = new AtomicReference<Throwable>();
2084    final Cell writeAgainCell1 = createCell(qf3, timestamp, seqId + 1, largeValue);
2085    final Cell writeAgainCell2 = createCell(qf4, timestamp, seqId + 1, largeValue);
2086    final int writeAgainCellByteSize =
2087      MutableSegment.getCellLength(writeAgainCell1) + MutableSegment.getCellLength(writeAgainCell2);
2088    final Thread writeAgainThread = new Thread(() -> {
2089      try {
2090        myCompactingMemStore.writeMemStoreAgainStartCyclicBarrier.await();
2091
2092        store.add(writeAgainCell1, new NonThreadSafeMemStoreSizing());
2093        store.add(writeAgainCell2, new NonThreadSafeMemStoreSizing());
2094
2095        myCompactingMemStore.writeMemStoreAgainEndCyclicBarrier.await();
2096      } catch (Throwable exception) {
2097        exceptionRef.set(exception);
2098      }
2099    });
2100    writeAgainThread.setName(MyCompactingMemStore5.WRITE_AGAIN_THREAD_NAME);
2101    writeAgainThread.start();
2102
2103    String oldThreadName = Thread.currentThread().getName();
2104    try {
2105      Thread.currentThread().setName(MyCompactingMemStore5.TAKE_SNAPSHOT_THREAD_NAME);
2106      /**
2107       * {@link CompactingMemStore#snapshot} must wait the in memory compact thread enters
2108       * {@link CompactingMemStore#flattenOneSegment},because {@link CompactingMemStore#snapshot}
2109       * would invoke {@link CompactingMemStore#stopCompaction}.
2110       */
2111      myCompactingMemStore.snapShotStartCyclicCyclicBarrier.await();
2112      MemStoreSnapshot memStoreSnapshot = myCompactingMemStore.snapshot();
2113      myCompactingMemStore.inMemoryCompactionEndCyclicBarrier.await();
2114      writeAgainThread.join();
2115
2116      assertTrue(memStoreSnapshot.getCellsCount() == 2);
2117      assertTrue(((int) (memStoreSnapshot.getDataSize())) == firstWriteCellByteSize);
2118      VersionedSegmentsList segments = myCompactingMemStore.getImmutableSegments();
2119      assertTrue(segments.getNumOfSegments() == 1);
2120      assertTrue(
2121        ((int) (segments.getStoreSegments().get(0).getDataSize())) == writeAgainCellByteSize);
2122      assertTrue(segments.getNumOfCells() == 2);
2123      assertTrue(myCompactingMemStore.setInMemoryCompactionFlagCounter.get() == 2);
2124      assertTrue(exceptionRef.get() == null);
2125      assertTrue(myCompactingMemStore.swapPipelineWithNullCounter.get() == 2);
2126    } finally {
2127      Thread.currentThread().setName(oldThreadName);
2128    }
2129  }
2130
2131  /**
2132   * <pre>
2133   * This test is for HBASE-26465,
2134   * test {@link DefaultMemStore#clearSnapshot} and {@link DefaultMemStore#getScanners} execute
2135   * concurrently. The threads sequence before HBASE-26465 is:
2136   * 1.The flush thread starts {@link DefaultMemStore} flushing after some cells have be added to
2137   *  {@link DefaultMemStore}.
2138   * 2.The flush thread stopping before {@link DefaultMemStore#clearSnapshot} in
2139   *   {@link HStore#updateStorefiles} after completed flushing memStore to hfile.
2140   * 3.The scan thread starts and stopping after {@link DefaultMemStore#getSnapshotSegments} in
2141   *   {@link DefaultMemStore#getScanners},here the scan thread gets the
2142   *   {@link DefaultMemStore#snapshot} which is created by the flush thread.
2143   * 4.The flush thread continues {@link DefaultMemStore#clearSnapshot} and close
2144   *   {@link DefaultMemStore#snapshot},because the reference count of the corresponding
2145   *   {@link MemStoreLABImpl} is 0, the {@link Chunk}s in corresponding {@link MemStoreLABImpl}
2146   *   are recycled.
2147   * 5.The scan thread continues {@link DefaultMemStore#getScanners},and create a
2148   *   {@link SegmentScanner} for this {@link DefaultMemStore#snapshot}, and increase the
2149   *   reference count of the corresponding {@link MemStoreLABImpl}, but {@link Chunk}s in
2150   *   corresponding {@link MemStoreLABImpl} are recycled by step 4, and these {@link Chunk}s may
2151   *   be overwritten by other write threads,which may cause serious problem.
2152   * After HBASE-26465,{@link DefaultMemStore#getScanners} and
2153   * {@link DefaultMemStore#clearSnapshot} could not execute concurrently.
2154   * </pre>
2155   */
2156  @Test
2157  public void testClearSnapshotGetScannerConcurrently() throws Exception {
2158    Configuration conf = HBaseConfiguration.create();
2159
2160    byte[] smallValue = new byte[3];
2161    byte[] largeValue = new byte[9];
2162    final long timestamp = EnvironmentEdgeManager.currentTime();
2163    final long seqId = 100;
2164    final Cell smallCell = createCell(qf1, timestamp, seqId, smallValue);
2165    final Cell largeCell = createCell(qf2, timestamp, seqId, largeValue);
2166    TreeSet<byte[]> quals = new TreeSet<>(Bytes.BYTES_COMPARATOR);
2167    quals.add(qf1);
2168    quals.add(qf2);
2169
2170    conf.set(HStore.MEMSTORE_CLASS_NAME, MyDefaultMemStore.class.getName());
2171    conf.setBoolean(WALFactory.WAL_ENABLED, false);
2172
2173    init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family).build());
2174    MyDefaultMemStore myDefaultMemStore = (MyDefaultMemStore) (store.memstore);
2175    myDefaultMemStore.store = store;
2176
2177    MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();
2178    store.add(smallCell, memStoreSizing);
2179    store.add(largeCell, memStoreSizing);
2180
2181    final AtomicReference<Throwable> exceptionRef = new AtomicReference<Throwable>();
2182    final Thread flushThread = new Thread(() -> {
2183      try {
2184        flushStore(store, id++);
2185      } catch (Throwable exception) {
2186        exceptionRef.set(exception);
2187      }
2188    });
2189    flushThread.setName(MyDefaultMemStore.FLUSH_THREAD_NAME);
2190    flushThread.start();
2191
2192    String oldThreadName = Thread.currentThread().getName();
2193    StoreScanner storeScanner = null;
2194    try {
2195      Thread.currentThread().setName(MyDefaultMemStore.GET_SCANNER_THREAD_NAME);
2196
2197      /**
2198       * Wait flush thread stopping before {@link DefaultMemStore#doClearSnapshot}
2199       */
2200      myDefaultMemStore.getScannerCyclicBarrier.await();
2201
2202      storeScanner = (StoreScanner) store.getScanner(new Scan(new Get(row)), quals, seqId + 1);
2203      flushThread.join();
2204
2205      if (myDefaultMemStore.shouldWait) {
2206        SegmentScanner segmentScanner = getTypeKeyValueScanner(storeScanner, SegmentScanner.class);
2207        MemStoreLABImpl memStoreLAB = (MemStoreLABImpl) (segmentScanner.segment.getMemStoreLAB());
2208        assertTrue(memStoreLAB.isClosed());
2209        assertTrue(!memStoreLAB.chunks.isEmpty());
2210        assertTrue(!memStoreLAB.isReclaimed());
2211
2212        Cell cell1 = segmentScanner.next();
2213        CellUtil.equals(smallCell, cell1);
2214        Cell cell2 = segmentScanner.next();
2215        CellUtil.equals(largeCell, cell2);
2216        assertNull(segmentScanner.next());
2217      } else {
2218        List<Cell> results = new ArrayList<>();
2219        storeScanner.next(results);
2220        assertEquals(2, results.size());
2221        CellUtil.equals(smallCell, results.get(0));
2222        CellUtil.equals(largeCell, results.get(1));
2223      }
2224      assertTrue(exceptionRef.get() == null);
2225    } finally {
2226      if (storeScanner != null) {
2227        storeScanner.close();
2228      }
2229      Thread.currentThread().setName(oldThreadName);
2230    }
2231  }
2232
2233  @SuppressWarnings("unchecked")
2234  private <T> T getTypeKeyValueScanner(StoreScanner storeScanner, Class<T> keyValueScannerClass) {
2235    List<T> resultScanners = new ArrayList<T>();
2236    for (KeyValueScanner keyValueScanner : storeScanner.currentScanners) {
2237      if (keyValueScannerClass.isInstance(keyValueScanner)) {
2238        resultScanners.add((T) keyValueScanner);
2239      }
2240    }
2241    assertTrue(resultScanners.size() == 1);
2242    return resultScanners.get(0);
2243  }
2244
2245  @Test
2246  public void testOnConfigurationChange() throws IOException {
2247    final int COMMON_MAX_FILES_TO_COMPACT = 10;
2248    final int NEW_COMMON_MAX_FILES_TO_COMPACT = 8;
2249    final int STORE_MAX_FILES_TO_COMPACT = 6;
2250
2251    // Build a table that its maxFileToCompact different from common configuration.
2252    Configuration conf = HBaseConfiguration.create();
2253    conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MAX_KEY,
2254      COMMON_MAX_FILES_TO_COMPACT);
2255    ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.newBuilder(family)
2256      .setConfiguration(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MAX_KEY,
2257        String.valueOf(STORE_MAX_FILES_TO_COMPACT))
2258      .build();
2259    init(this.name.getMethodName(), conf, hcd);
2260
2261    // After updating common configuration, the conf in HStore itself must not be changed.
2262    conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MAX_KEY,
2263      NEW_COMMON_MAX_FILES_TO_COMPACT);
2264    this.store.onConfigurationChange(conf);
2265    assertEquals(STORE_MAX_FILES_TO_COMPACT,
2266      store.getStoreEngine().getCompactionPolicy().getConf().getMaxFilesToCompact());
2267  }
2268
2269  /**
2270   * This test is for HBASE-26476
2271   */
2272  @Test
2273  public void testExtendsDefaultMemStore() throws Exception {
2274    Configuration conf = HBaseConfiguration.create();
2275    conf.setBoolean(WALFactory.WAL_ENABLED, false);
2276
2277    init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family).build());
2278    assertTrue(this.store.memstore.getClass() == DefaultMemStore.class);
2279    tearDown();
2280
2281    conf.set(HStore.MEMSTORE_CLASS_NAME, CustomDefaultMemStore.class.getName());
2282    init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family).build());
2283    assertTrue(this.store.memstore.getClass() == CustomDefaultMemStore.class);
2284  }
2285
2286  static class CustomDefaultMemStore extends DefaultMemStore {
2287
2288    public CustomDefaultMemStore(Configuration conf, CellComparator c,
2289      RegionServicesForStores regionServices) {
2290      super(conf, c, regionServices);
2291    }
2292
2293  }
2294
2295  /**
2296   * This test is for HBASE-26488
2297   */
2298  @Test
2299  public void testMemoryLeakWhenFlushMemStoreRetrying() throws Exception {
2300
2301    Configuration conf = HBaseConfiguration.create();
2302
2303    byte[] smallValue = new byte[3];
2304    byte[] largeValue = new byte[9];
2305    final long timestamp = EnvironmentEdgeManager.currentTime();
2306    final long seqId = 100;
2307    final Cell smallCell = createCell(qf1, timestamp, seqId, smallValue);
2308    final Cell largeCell = createCell(qf2, timestamp, seqId, largeValue);
2309    TreeSet<byte[]> quals = new TreeSet<>(Bytes.BYTES_COMPARATOR);
2310    quals.add(qf1);
2311    quals.add(qf2);
2312
2313    conf.set(HStore.MEMSTORE_CLASS_NAME, MyDefaultMemStore1.class.getName());
2314    conf.setBoolean(WALFactory.WAL_ENABLED, false);
2315    conf.set(DefaultStoreEngine.DEFAULT_STORE_FLUSHER_CLASS_KEY,
2316      MyDefaultStoreFlusher.class.getName());
2317
2318    init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family).build());
2319    MyDefaultMemStore1 myDefaultMemStore = (MyDefaultMemStore1) (store.memstore);
2320    assertTrue((store.storeEngine.getStoreFlusher()) instanceof MyDefaultStoreFlusher);
2321
2322    MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();
2323    store.add(smallCell, memStoreSizing);
2324    store.add(largeCell, memStoreSizing);
2325    flushStore(store, id++);
2326
2327    MemStoreLABImpl memStoreLAB =
2328      (MemStoreLABImpl) (myDefaultMemStore.snapshotImmutableSegment.getMemStoreLAB());
2329    assertTrue(memStoreLAB.isClosed());
2330    assertTrue(memStoreLAB.getRefCntValue() == 0);
2331    assertTrue(memStoreLAB.isReclaimed());
2332    assertTrue(memStoreLAB.chunks.isEmpty());
2333    StoreScanner storeScanner = null;
2334    try {
2335      storeScanner = (StoreScanner) store.getScanner(new Scan(new Get(row)), quals, seqId + 1);
2336      assertTrue(store.storeEngine.getStoreFileManager().getStorefileCount() == 1);
2337      assertTrue(store.memstore.size().getCellsCount() == 0);
2338      assertTrue(store.memstore.getSnapshotSize().getCellsCount() == 0);
2339      assertTrue(storeScanner.currentScanners.size() == 1);
2340      assertTrue(storeScanner.currentScanners.get(0) instanceof StoreFileScanner);
2341
2342      List<Cell> results = new ArrayList<>();
2343      storeScanner.next(results);
2344      assertEquals(2, results.size());
2345      CellUtil.equals(smallCell, results.get(0));
2346      CellUtil.equals(largeCell, results.get(1));
2347    } finally {
2348      if (storeScanner != null) {
2349        storeScanner.close();
2350      }
2351    }
2352  }
2353
2354  static class MyDefaultMemStore1 extends DefaultMemStore {
2355
2356    private ImmutableSegment snapshotImmutableSegment;
2357
2358    public MyDefaultMemStore1(Configuration conf, CellComparator c,
2359      RegionServicesForStores regionServices) {
2360      super(conf, c, regionServices);
2361    }
2362
2363    @Override
2364    public MemStoreSnapshot snapshot() {
2365      MemStoreSnapshot result = super.snapshot();
2366      this.snapshotImmutableSegment = snapshot;
2367      return result;
2368    }
2369
2370  }
2371
2372  public static class MyDefaultStoreFlusher extends DefaultStoreFlusher {
2373    private static final AtomicInteger failCounter = new AtomicInteger(1);
2374    private static final AtomicInteger counter = new AtomicInteger(0);
2375
2376    public MyDefaultStoreFlusher(Configuration conf, HStore store) {
2377      super(conf, store);
2378    }
2379
2380    @Override
2381    public List<Path> flushSnapshot(MemStoreSnapshot snapshot, long cacheFlushId,
2382      MonitoredTask status, ThroughputController throughputController,
2383      FlushLifeCycleTracker tracker, Consumer<Path> writerCreationTracker) throws IOException {
2384      counter.incrementAndGet();
2385      return super.flushSnapshot(snapshot, cacheFlushId, status, throughputController, tracker,
2386        writerCreationTracker);
2387    }
2388
2389    @Override
2390    protected void performFlush(InternalScanner scanner, final CellSink sink,
2391      ThroughputController throughputController) throws IOException {
2392
2393      final int currentCount = counter.get();
2394      CellSink newCellSink = (cell) -> {
2395        if (currentCount <= failCounter.get()) {
2396          throw new IOException("Simulated exception by tests");
2397        }
2398        sink.append(cell);
2399      };
2400      super.performFlush(scanner, newCellSink, throughputController);
2401    }
2402  }
2403
2404  /**
2405   * This test is for HBASE-26494, test the {@link RefCnt} behaviors in {@link ImmutableMemStoreLAB}
2406   */
2407  @Test
2408  public void testImmutableMemStoreLABRefCnt() throws Exception {
2409    Configuration conf = HBaseConfiguration.create();
2410
2411    byte[] smallValue = new byte[3];
2412    byte[] largeValue = new byte[9];
2413    final long timestamp = EnvironmentEdgeManager.currentTime();
2414    final long seqId = 100;
2415    final Cell smallCell1 = createCell(qf1, timestamp, seqId, smallValue);
2416    final Cell largeCell1 = createCell(qf2, timestamp, seqId, largeValue);
2417    final Cell smallCell2 = createCell(qf3, timestamp, seqId + 1, smallValue);
2418    final Cell largeCell2 = createCell(qf4, timestamp, seqId + 1, largeValue);
2419    final Cell smallCell3 = createCell(qf5, timestamp, seqId + 2, smallValue);
2420    final Cell largeCell3 = createCell(qf6, timestamp, seqId + 2, largeValue);
2421
2422    int smallCellByteSize = MutableSegment.getCellLength(smallCell1);
2423    int largeCellByteSize = MutableSegment.getCellLength(largeCell1);
2424    int firstWriteCellByteSize = (smallCellByteSize + largeCellByteSize);
2425    int flushByteSize = firstWriteCellByteSize - 2;
2426
2427    // set CompactingMemStore.inmemoryFlushSize to flushByteSize.
2428    conf.set(HStore.MEMSTORE_CLASS_NAME, CompactingMemStore.class.getName());
2429    conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.005);
2430    conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(flushByteSize * 200));
2431    conf.setBoolean(WALFactory.WAL_ENABLED, false);
2432
2433    init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)
2434      .setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());
2435
2436    final CompactingMemStore myCompactingMemStore = ((CompactingMemStore) store.memstore);
2437    assertTrue((int) (myCompactingMemStore.getInmemoryFlushSize()) == flushByteSize);
2438    myCompactingMemStore.allowCompaction.set(false);
2439
2440    NonThreadSafeMemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();
2441    store.add(smallCell1, memStoreSizing);
2442    store.add(largeCell1, memStoreSizing);
2443    store.add(smallCell2, memStoreSizing);
2444    store.add(largeCell2, memStoreSizing);
2445    store.add(smallCell3, memStoreSizing);
2446    store.add(largeCell3, memStoreSizing);
2447    VersionedSegmentsList versionedSegmentsList = myCompactingMemStore.getImmutableSegments();
2448    assertTrue(versionedSegmentsList.getNumOfSegments() == 3);
2449    List<ImmutableSegment> segments = versionedSegmentsList.getStoreSegments();
2450    List<MemStoreLABImpl> memStoreLABs = new ArrayList<MemStoreLABImpl>(segments.size());
2451    for (ImmutableSegment segment : segments) {
2452      memStoreLABs.add((MemStoreLABImpl) segment.getMemStoreLAB());
2453    }
2454    List<KeyValueScanner> scanners1 = myCompactingMemStore.getScanners(Long.MAX_VALUE);
2455    for (MemStoreLABImpl memStoreLAB : memStoreLABs) {
2456      assertTrue(memStoreLAB.getRefCntValue() == 2);
2457    }
2458
2459    myCompactingMemStore.allowCompaction.set(true);
2460    myCompactingMemStore.flushInMemory();
2461
2462    versionedSegmentsList = myCompactingMemStore.getImmutableSegments();
2463    assertTrue(versionedSegmentsList.getNumOfSegments() == 1);
2464    ImmutableMemStoreLAB immutableMemStoreLAB =
2465      (ImmutableMemStoreLAB) (versionedSegmentsList.getStoreSegments().get(0).getMemStoreLAB());
2466    for (MemStoreLABImpl memStoreLAB : memStoreLABs) {
2467      assertTrue(memStoreLAB.getRefCntValue() == 2);
2468    }
2469
2470    List<KeyValueScanner> scanners2 = myCompactingMemStore.getScanners(Long.MAX_VALUE);
2471    for (MemStoreLABImpl memStoreLAB : memStoreLABs) {
2472      assertTrue(memStoreLAB.getRefCntValue() == 2);
2473    }
2474    assertTrue(immutableMemStoreLAB.getRefCntValue() == 2);
2475    for (KeyValueScanner scanner : scanners1) {
2476      scanner.close();
2477    }
2478    for (MemStoreLABImpl memStoreLAB : memStoreLABs) {
2479      assertTrue(memStoreLAB.getRefCntValue() == 1);
2480    }
2481    for (KeyValueScanner scanner : scanners2) {
2482      scanner.close();
2483    }
2484    for (MemStoreLABImpl memStoreLAB : memStoreLABs) {
2485      assertTrue(memStoreLAB.getRefCntValue() == 1);
2486    }
2487    assertTrue(immutableMemStoreLAB.getRefCntValue() == 1);
2488    flushStore(store, id++);
2489    for (MemStoreLABImpl memStoreLAB : memStoreLABs) {
2490      assertTrue(memStoreLAB.getRefCntValue() == 0);
2491    }
2492    assertTrue(immutableMemStoreLAB.getRefCntValue() == 0);
2493    assertTrue(immutableMemStoreLAB.isClosed());
2494    for (MemStoreLABImpl memStoreLAB : memStoreLABs) {
2495      assertTrue(memStoreLAB.isClosed());
2496      assertTrue(memStoreLAB.isReclaimed());
2497      assertTrue(memStoreLAB.chunks.isEmpty());
2498    }
2499  }
2500
2501  private HStoreFile mockStoreFileWithLength(long length) {
2502    HStoreFile sf = mock(HStoreFile.class);
2503    StoreFileReader sfr = mock(StoreFileReader.class);
2504    when(sf.isHFile()).thenReturn(true);
2505    when(sf.getReader()).thenReturn(sfr);
2506    when(sfr.length()).thenReturn(length);
2507    return sf;
2508  }
2509
2510  private static class MyThread extends Thread {
2511    private StoreScanner scanner;
2512    private KeyValueHeap heap;
2513
2514    public MyThread(StoreScanner scanner) {
2515      this.scanner = scanner;
2516    }
2517
2518    public KeyValueHeap getHeap() {
2519      return this.heap;
2520    }
2521
2522    @Override
2523    public void run() {
2524      scanner.trySwitchToStreamRead();
2525      heap = scanner.heap;
2526    }
2527  }
2528
2529  private static class MyMemStoreCompactor extends MemStoreCompactor {
2530    private static final AtomicInteger RUNNER_COUNT = new AtomicInteger(0);
2531    private static final CountDownLatch START_COMPACTOR_LATCH = new CountDownLatch(1);
2532
2533    public MyMemStoreCompactor(CompactingMemStore compactingMemStore,
2534      MemoryCompactionPolicy compactionPolicy) throws IllegalArgumentIOException {
2535      super(compactingMemStore, compactionPolicy);
2536    }
2537
2538    @Override
2539    public boolean start() throws IOException {
2540      boolean isFirst = RUNNER_COUNT.getAndIncrement() == 0;
2541      if (isFirst) {
2542        try {
2543          START_COMPACTOR_LATCH.await();
2544          return super.start();
2545        } catch (InterruptedException ex) {
2546          throw new RuntimeException(ex);
2547        }
2548      }
2549      return super.start();
2550    }
2551  }
2552
2553  public static class MyCompactingMemStoreWithCustomCompactor extends CompactingMemStore {
2554    private static final AtomicInteger RUNNER_COUNT = new AtomicInteger(0);
2555
2556    public MyCompactingMemStoreWithCustomCompactor(Configuration conf, CellComparatorImpl c,
2557      HStore store, RegionServicesForStores regionServices, MemoryCompactionPolicy compactionPolicy)
2558      throws IOException {
2559      super(conf, c, store, regionServices, compactionPolicy);
2560    }
2561
2562    @Override
2563    protected MemStoreCompactor createMemStoreCompactor(MemoryCompactionPolicy compactionPolicy)
2564      throws IllegalArgumentIOException {
2565      return new MyMemStoreCompactor(this, compactionPolicy);
2566    }
2567
2568    @Override
2569    protected boolean setInMemoryCompactionFlag() {
2570      boolean rval = super.setInMemoryCompactionFlag();
2571      if (rval) {
2572        RUNNER_COUNT.incrementAndGet();
2573        if (LOG.isDebugEnabled()) {
2574          LOG.debug("runner count: " + RUNNER_COUNT.get());
2575        }
2576      }
2577      return rval;
2578    }
2579  }
2580
2581  public static class MyCompactingMemStore extends CompactingMemStore {
2582    private static final AtomicBoolean START_TEST = new AtomicBoolean(false);
2583    private final CountDownLatch getScannerLatch = new CountDownLatch(1);
2584    private final CountDownLatch snapshotLatch = new CountDownLatch(1);
2585
2586    public MyCompactingMemStore(Configuration conf, CellComparatorImpl c, HStore store,
2587      RegionServicesForStores regionServices, MemoryCompactionPolicy compactionPolicy)
2588      throws IOException {
2589      super(conf, c, store, regionServices, compactionPolicy);
2590    }
2591
2592    @Override
2593    protected List<KeyValueScanner> createList(int capacity) {
2594      if (START_TEST.get()) {
2595        try {
2596          getScannerLatch.countDown();
2597          snapshotLatch.await();
2598        } catch (InterruptedException e) {
2599          throw new RuntimeException(e);
2600        }
2601      }
2602      return new ArrayList<>(capacity);
2603    }
2604
2605    @Override
2606    protected void pushActiveToPipeline(MutableSegment active, boolean checkEmpty) {
2607      if (START_TEST.get()) {
2608        try {
2609          getScannerLatch.await();
2610        } catch (InterruptedException e) {
2611          throw new RuntimeException(e);
2612        }
2613      }
2614
2615      super.pushActiveToPipeline(active, checkEmpty);
2616      if (START_TEST.get()) {
2617        snapshotLatch.countDown();
2618      }
2619    }
2620  }
2621
2622  interface MyListHook {
2623    void hook(int currentSize);
2624  }
2625
2626  private static class MyList<T> implements List<T> {
2627    private final List<T> delegatee = new ArrayList<>();
2628    private final MyListHook hookAtAdd;
2629
2630    MyList(final MyListHook hookAtAdd) {
2631      this.hookAtAdd = hookAtAdd;
2632    }
2633
2634    @Override
2635    public int size() {
2636      return delegatee.size();
2637    }
2638
2639    @Override
2640    public boolean isEmpty() {
2641      return delegatee.isEmpty();
2642    }
2643
2644    @Override
2645    public boolean contains(Object o) {
2646      return delegatee.contains(o);
2647    }
2648
2649    @Override
2650    public Iterator<T> iterator() {
2651      return delegatee.iterator();
2652    }
2653
2654    @Override
2655    public Object[] toArray() {
2656      return delegatee.toArray();
2657    }
2658
2659    @Override
2660    public <R> R[] toArray(R[] a) {
2661      return delegatee.toArray(a);
2662    }
2663
2664    @Override
2665    public boolean add(T e) {
2666      hookAtAdd.hook(size());
2667      return delegatee.add(e);
2668    }
2669
2670    @Override
2671    public boolean remove(Object o) {
2672      return delegatee.remove(o);
2673    }
2674
2675    @Override
2676    public boolean containsAll(Collection<?> c) {
2677      return delegatee.containsAll(c);
2678    }
2679
2680    @Override
2681    public boolean addAll(Collection<? extends T> c) {
2682      return delegatee.addAll(c);
2683    }
2684
2685    @Override
2686    public boolean addAll(int index, Collection<? extends T> c) {
2687      return delegatee.addAll(index, c);
2688    }
2689
2690    @Override
2691    public boolean removeAll(Collection<?> c) {
2692      return delegatee.removeAll(c);
2693    }
2694
2695    @Override
2696    public boolean retainAll(Collection<?> c) {
2697      return delegatee.retainAll(c);
2698    }
2699
2700    @Override
2701    public void clear() {
2702      delegatee.clear();
2703    }
2704
2705    @Override
2706    public T get(int index) {
2707      return delegatee.get(index);
2708    }
2709
2710    @Override
2711    public T set(int index, T element) {
2712      return delegatee.set(index, element);
2713    }
2714
2715    @Override
2716    public void add(int index, T element) {
2717      delegatee.add(index, element);
2718    }
2719
2720    @Override
2721    public T remove(int index) {
2722      return delegatee.remove(index);
2723    }
2724
2725    @Override
2726    public int indexOf(Object o) {
2727      return delegatee.indexOf(o);
2728    }
2729
2730    @Override
2731    public int lastIndexOf(Object o) {
2732      return delegatee.lastIndexOf(o);
2733    }
2734
2735    @Override
2736    public ListIterator<T> listIterator() {
2737      return delegatee.listIterator();
2738    }
2739
2740    @Override
2741    public ListIterator<T> listIterator(int index) {
2742      return delegatee.listIterator(index);
2743    }
2744
2745    @Override
2746    public List<T> subList(int fromIndex, int toIndex) {
2747      return delegatee.subList(fromIndex, toIndex);
2748    }
2749  }
2750
2751  public static class MyCompactingMemStore2 extends CompactingMemStore {
2752    private static final String LARGE_CELL_THREAD_NAME = "largeCellThread";
2753    private static final String SMALL_CELL_THREAD_NAME = "smallCellThread";
2754    private final CyclicBarrier preCyclicBarrier = new CyclicBarrier(2);
2755    private final CyclicBarrier postCyclicBarrier = new CyclicBarrier(2);
2756    private final AtomicInteger largeCellPreUpdateCounter = new AtomicInteger(0);
2757    private final AtomicInteger smallCellPreUpdateCounter = new AtomicInteger(0);
2758
2759    public MyCompactingMemStore2(Configuration conf, CellComparatorImpl cellComparator,
2760      HStore store, RegionServicesForStores regionServices, MemoryCompactionPolicy compactionPolicy)
2761      throws IOException {
2762      super(conf, cellComparator, store, regionServices, compactionPolicy);
2763    }
2764
2765    @Override
2766    protected boolean checkAndAddToActiveSize(MutableSegment currActive, Cell cellToAdd,
2767      MemStoreSizing memstoreSizing) {
2768      if (Thread.currentThread().getName().equals(LARGE_CELL_THREAD_NAME)) {
2769        int currentCount = largeCellPreUpdateCounter.incrementAndGet();
2770        if (currentCount <= 1) {
2771          try {
2772            /**
2773             * smallCellThread enters CompactingMemStore.checkAndAddToActiveSize first, then
2774             * largeCellThread enters CompactingMemStore.checkAndAddToActiveSize, and then
2775             * largeCellThread invokes flushInMemory.
2776             */
2777            preCyclicBarrier.await();
2778          } catch (Throwable e) {
2779            throw new RuntimeException(e);
2780          }
2781        }
2782      }
2783
2784      boolean returnValue = super.checkAndAddToActiveSize(currActive, cellToAdd, memstoreSizing);
2785      if (Thread.currentThread().getName().equals(SMALL_CELL_THREAD_NAME)) {
2786        try {
2787          preCyclicBarrier.await();
2788        } catch (Throwable e) {
2789          throw new RuntimeException(e);
2790        }
2791      }
2792      return returnValue;
2793    }
2794
2795    @Override
2796    protected void doAdd(MutableSegment currentActive, Cell cell, MemStoreSizing memstoreSizing) {
2797      if (Thread.currentThread().getName().equals(SMALL_CELL_THREAD_NAME)) {
2798        try {
2799          /**
2800           * After largeCellThread finished flushInMemory method, smallCellThread can add cell to
2801           * currentActive . That is to say when largeCellThread called flushInMemory method,
2802           * currentActive has no cell.
2803           */
2804          postCyclicBarrier.await();
2805        } catch (Throwable e) {
2806          throw new RuntimeException(e);
2807        }
2808      }
2809      super.doAdd(currentActive, cell, memstoreSizing);
2810    }
2811
2812    @Override
2813    protected void flushInMemory(MutableSegment currentActiveMutableSegment) {
2814      super.flushInMemory(currentActiveMutableSegment);
2815      if (Thread.currentThread().getName().equals(LARGE_CELL_THREAD_NAME)) {
2816        if (largeCellPreUpdateCounter.get() <= 1) {
2817          try {
2818            postCyclicBarrier.await();
2819          } catch (Throwable e) {
2820            throw new RuntimeException(e);
2821          }
2822        }
2823      }
2824    }
2825
2826  }
2827
2828  public static class MyCompactingMemStore3 extends CompactingMemStore {
2829    private static final String LARGE_CELL_THREAD_NAME = "largeCellThread";
2830    private static final String SMALL_CELL_THREAD_NAME = "smallCellThread";
2831
2832    private final CyclicBarrier preCyclicBarrier = new CyclicBarrier(2);
2833    private final CyclicBarrier postCyclicBarrier = new CyclicBarrier(2);
2834    private final AtomicInteger flushCounter = new AtomicInteger(0);
2835    private static final int CELL_COUNT = 5;
2836    private boolean flushByteSizeLessThanSmallAndLargeCellSize = true;
2837
2838    public MyCompactingMemStore3(Configuration conf, CellComparatorImpl cellComparator,
2839      HStore store, RegionServicesForStores regionServices, MemoryCompactionPolicy compactionPolicy)
2840      throws IOException {
2841      super(conf, cellComparator, store, regionServices, compactionPolicy);
2842    }
2843
2844    @Override
2845    protected boolean checkAndAddToActiveSize(MutableSegment currActive, Cell cellToAdd,
2846      MemStoreSizing memstoreSizing) {
2847      if (!flushByteSizeLessThanSmallAndLargeCellSize) {
2848        return super.checkAndAddToActiveSize(currActive, cellToAdd, memstoreSizing);
2849      }
2850      if (Thread.currentThread().getName().equals(LARGE_CELL_THREAD_NAME)) {
2851        try {
2852          preCyclicBarrier.await();
2853        } catch (Throwable e) {
2854          throw new RuntimeException(e);
2855        }
2856      }
2857
2858      boolean returnValue = super.checkAndAddToActiveSize(currActive, cellToAdd, memstoreSizing);
2859      if (Thread.currentThread().getName().equals(SMALL_CELL_THREAD_NAME)) {
2860        try {
2861          preCyclicBarrier.await();
2862        } catch (Throwable e) {
2863          throw new RuntimeException(e);
2864        }
2865      }
2866      return returnValue;
2867    }
2868
2869    @Override
2870    protected void postUpdate(MutableSegment currentActiveMutableSegment) {
2871      super.postUpdate(currentActiveMutableSegment);
2872      if (!flushByteSizeLessThanSmallAndLargeCellSize) {
2873        try {
2874          postCyclicBarrier.await();
2875        } catch (Throwable e) {
2876          throw new RuntimeException(e);
2877        }
2878        return;
2879      }
2880
2881      if (Thread.currentThread().getName().equals(SMALL_CELL_THREAD_NAME)) {
2882        try {
2883          postCyclicBarrier.await();
2884        } catch (Throwable e) {
2885          throw new RuntimeException(e);
2886        }
2887      }
2888    }
2889
2890    @Override
2891    protected void flushInMemory(MutableSegment currentActiveMutableSegment) {
2892      super.flushInMemory(currentActiveMutableSegment);
2893      flushCounter.incrementAndGet();
2894      if (!flushByteSizeLessThanSmallAndLargeCellSize) {
2895        return;
2896      }
2897
2898      assertTrue(Thread.currentThread().getName().equals(LARGE_CELL_THREAD_NAME));
2899      try {
2900        postCyclicBarrier.await();
2901      } catch (Throwable e) {
2902        throw new RuntimeException(e);
2903      }
2904
2905    }
2906
2907    void disableCompaction() {
2908      allowCompaction.set(false);
2909    }
2910
2911    void enableCompaction() {
2912      allowCompaction.set(true);
2913    }
2914
2915  }
2916
2917  public static class MyCompactingMemStore4 extends CompactingMemStore {
2918    private static final String TAKE_SNAPSHOT_THREAD_NAME = "takeSnapShotThread";
2919    /**
2920     * {@link CompactingMemStore#flattenOneSegment} must execute after
2921     * {@link CompactingMemStore#getImmutableSegments}
2922     */
2923    private final CyclicBarrier flattenOneSegmentPreCyclicBarrier = new CyclicBarrier(2);
2924    /**
2925     * Only after {@link CompactingMemStore#flattenOneSegment} completed,
2926     * {@link CompactingMemStore#swapPipelineWithNull} could execute.
2927     */
2928    private final CyclicBarrier flattenOneSegmentPostCyclicBarrier = new CyclicBarrier(2);
2929    /**
2930     * Only the in memory compact thread enters {@link CompactingMemStore#flattenOneSegment},the
2931     * snapshot thread starts {@link CompactingMemStore#snapshot},because
2932     * {@link CompactingMemStore#snapshot} would invoke {@link CompactingMemStore#stopCompaction}.
2933     */
2934    private final CyclicBarrier snapShotStartCyclicCyclicBarrier = new CyclicBarrier(2);
2935    /**
2936     * To wait for {@link CompactingMemStore.InMemoryCompactionRunnable} stopping.
2937     */
2938    private final CyclicBarrier inMemoryCompactionEndCyclicBarrier = new CyclicBarrier(2);
2939    private final AtomicInteger getImmutableSegmentsListCounter = new AtomicInteger(0);
2940    private final AtomicInteger swapPipelineWithNullCounter = new AtomicInteger(0);
2941    private final AtomicInteger flattenOneSegmentCounter = new AtomicInteger(0);
2942    private final AtomicInteger setInMemoryCompactionFlagCounter = new AtomicInteger(0);
2943
2944    public MyCompactingMemStore4(Configuration conf, CellComparatorImpl cellComparator,
2945      HStore store, RegionServicesForStores regionServices, MemoryCompactionPolicy compactionPolicy)
2946      throws IOException {
2947      super(conf, cellComparator, store, regionServices, compactionPolicy);
2948    }
2949
2950    @Override
2951    public VersionedSegmentsList getImmutableSegments() {
2952      VersionedSegmentsList result = super.getImmutableSegments();
2953      if (Thread.currentThread().getName().equals(TAKE_SNAPSHOT_THREAD_NAME)) {
2954        int currentCount = getImmutableSegmentsListCounter.incrementAndGet();
2955        if (currentCount <= 1) {
2956          try {
2957            flattenOneSegmentPreCyclicBarrier.await();
2958          } catch (Throwable e) {
2959            throw new RuntimeException(e);
2960          }
2961        }
2962      }
2963      return result;
2964    }
2965
2966    @Override
2967    protected boolean swapPipelineWithNull(VersionedSegmentsList segments) {
2968      if (Thread.currentThread().getName().equals(TAKE_SNAPSHOT_THREAD_NAME)) {
2969        int currentCount = swapPipelineWithNullCounter.incrementAndGet();
2970        if (currentCount <= 1) {
2971          try {
2972            flattenOneSegmentPostCyclicBarrier.await();
2973          } catch (Throwable e) {
2974            throw new RuntimeException(e);
2975          }
2976        }
2977      }
2978      boolean result = super.swapPipelineWithNull(segments);
2979      if (Thread.currentThread().getName().equals(TAKE_SNAPSHOT_THREAD_NAME)) {
2980        int currentCount = swapPipelineWithNullCounter.get();
2981        if (currentCount <= 1) {
2982          assertTrue(!result);
2983        }
2984        if (currentCount == 2) {
2985          assertTrue(result);
2986        }
2987      }
2988      return result;
2989
2990    }
2991
2992    @Override
2993    public void flattenOneSegment(long requesterVersion, Action action) {
2994      int currentCount = flattenOneSegmentCounter.incrementAndGet();
2995      if (currentCount <= 1) {
2996        try {
2997          /**
2998           * {@link CompactingMemStore#snapshot} could start.
2999           */
3000          snapShotStartCyclicCyclicBarrier.await();
3001          flattenOneSegmentPreCyclicBarrier.await();
3002        } catch (Throwable e) {
3003          throw new RuntimeException(e);
3004        }
3005      }
3006      super.flattenOneSegment(requesterVersion, action);
3007      if (currentCount <= 1) {
3008        try {
3009          flattenOneSegmentPostCyclicBarrier.await();
3010        } catch (Throwable e) {
3011          throw new RuntimeException(e);
3012        }
3013      }
3014    }
3015
3016    @Override
3017    protected boolean setInMemoryCompactionFlag() {
3018      boolean result = super.setInMemoryCompactionFlag();
3019      assertTrue(result);
3020      setInMemoryCompactionFlagCounter.incrementAndGet();
3021      return result;
3022    }
3023
3024    @Override
3025    void inMemoryCompaction() {
3026      try {
3027        super.inMemoryCompaction();
3028      } finally {
3029        try {
3030          inMemoryCompactionEndCyclicBarrier.await();
3031        } catch (Throwable e) {
3032          throw new RuntimeException(e);
3033        }
3034
3035      }
3036    }
3037
3038  }
3039
3040  public static class MyCompactingMemStore5 extends CompactingMemStore {
3041    private static final String TAKE_SNAPSHOT_THREAD_NAME = "takeSnapShotThread";
3042    private static final String WRITE_AGAIN_THREAD_NAME = "writeAgainThread";
3043    /**
3044     * {@link CompactingMemStore#flattenOneSegment} must execute after
3045     * {@link CompactingMemStore#getImmutableSegments}
3046     */
3047    private final CyclicBarrier flattenOneSegmentPreCyclicBarrier = new CyclicBarrier(2);
3048    /**
3049     * Only after {@link CompactingMemStore#flattenOneSegment} completed,
3050     * {@link CompactingMemStore#swapPipelineWithNull} could execute.
3051     */
3052    private final CyclicBarrier flattenOneSegmentPostCyclicBarrier = new CyclicBarrier(2);
3053    /**
3054     * Only the in memory compact thread enters {@link CompactingMemStore#flattenOneSegment},the
3055     * snapshot thread starts {@link CompactingMemStore#snapshot},because
3056     * {@link CompactingMemStore#snapshot} would invoke {@link CompactingMemStore#stopCompaction}.
3057     */
3058    private final CyclicBarrier snapShotStartCyclicCyclicBarrier = new CyclicBarrier(2);
3059    /**
3060     * To wait for {@link CompactingMemStore.InMemoryCompactionRunnable} stopping.
3061     */
3062    private final CyclicBarrier inMemoryCompactionEndCyclicBarrier = new CyclicBarrier(2);
3063    private final AtomicInteger getImmutableSegmentsListCounter = new AtomicInteger(0);
3064    private final AtomicInteger swapPipelineWithNullCounter = new AtomicInteger(0);
3065    private final AtomicInteger flattenOneSegmentCounter = new AtomicInteger(0);
3066    private final AtomicInteger setInMemoryCompactionFlagCounter = new AtomicInteger(0);
3067    /**
3068     * Only the snapshot thread retry {@link CompactingMemStore#swapPipelineWithNull}, writeAgain
3069     * thread could start.
3070     */
3071    private final CyclicBarrier writeMemStoreAgainStartCyclicBarrier = new CyclicBarrier(2);
3072    /**
3073     * This is used for snapshot thread,writeAgain thread and in memory compact thread. Only the
3074     * writeAgain thread completes, {@link CompactingMemStore#swapPipelineWithNull} would
3075     * execute,and in memory compact thread would exit,because we expect that in memory compact
3076     * executing only once.
3077     */
3078    private final CyclicBarrier writeMemStoreAgainEndCyclicBarrier = new CyclicBarrier(3);
3079
3080    public MyCompactingMemStore5(Configuration conf, CellComparatorImpl cellComparator,
3081      HStore store, RegionServicesForStores regionServices, MemoryCompactionPolicy compactionPolicy)
3082      throws IOException {
3083      super(conf, cellComparator, store, regionServices, compactionPolicy);
3084    }
3085
3086    @Override
3087    public VersionedSegmentsList getImmutableSegments() {
3088      VersionedSegmentsList result = super.getImmutableSegments();
3089      if (Thread.currentThread().getName().equals(TAKE_SNAPSHOT_THREAD_NAME)) {
3090        int currentCount = getImmutableSegmentsListCounter.incrementAndGet();
3091        if (currentCount <= 1) {
3092          try {
3093            flattenOneSegmentPreCyclicBarrier.await();
3094          } catch (Throwable e) {
3095            throw new RuntimeException(e);
3096          }
3097        }
3098
3099      }
3100
3101      return result;
3102    }
3103
3104    @Override
3105    protected boolean swapPipelineWithNull(VersionedSegmentsList segments) {
3106      if (Thread.currentThread().getName().equals(TAKE_SNAPSHOT_THREAD_NAME)) {
3107        int currentCount = swapPipelineWithNullCounter.incrementAndGet();
3108        if (currentCount <= 1) {
3109          try {
3110            flattenOneSegmentPostCyclicBarrier.await();
3111          } catch (Throwable e) {
3112            throw new RuntimeException(e);
3113          }
3114        }
3115
3116        if (currentCount == 2) {
3117          try {
3118            /**
3119             * Only the snapshot thread retry {@link CompactingMemStore#swapPipelineWithNull},
3120             * writeAgain thread could start.
3121             */
3122            writeMemStoreAgainStartCyclicBarrier.await();
3123            /**
3124             * Only the writeAgain thread completes, retry
3125             * {@link CompactingMemStore#swapPipelineWithNull} would execute.
3126             */
3127            writeMemStoreAgainEndCyclicBarrier.await();
3128          } catch (Throwable e) {
3129            throw new RuntimeException(e);
3130          }
3131        }
3132
3133      }
3134      boolean result = super.swapPipelineWithNull(segments);
3135      if (Thread.currentThread().getName().equals(TAKE_SNAPSHOT_THREAD_NAME)) {
3136        int currentCount = swapPipelineWithNullCounter.get();
3137        if (currentCount <= 1) {
3138          assertTrue(!result);
3139        }
3140        if (currentCount == 2) {
3141          assertTrue(result);
3142        }
3143      }
3144      return result;
3145
3146    }
3147
3148    @Override
3149    public void flattenOneSegment(long requesterVersion, Action action) {
3150      int currentCount = flattenOneSegmentCounter.incrementAndGet();
3151      if (currentCount <= 1) {
3152        try {
3153          /**
3154           * {@link CompactingMemStore#snapshot} could start.
3155           */
3156          snapShotStartCyclicCyclicBarrier.await();
3157          flattenOneSegmentPreCyclicBarrier.await();
3158        } catch (Throwable e) {
3159          throw new RuntimeException(e);
3160        }
3161      }
3162      super.flattenOneSegment(requesterVersion, action);
3163      if (currentCount <= 1) {
3164        try {
3165          flattenOneSegmentPostCyclicBarrier.await();
3166          /**
3167           * Only the writeAgain thread completes, in memory compact thread would exit,because we
3168           * expect that in memory compact executing only once.
3169           */
3170          writeMemStoreAgainEndCyclicBarrier.await();
3171        } catch (Throwable e) {
3172          throw new RuntimeException(e);
3173        }
3174
3175      }
3176    }
3177
3178    @Override
3179    protected boolean setInMemoryCompactionFlag() {
3180      boolean result = super.setInMemoryCompactionFlag();
3181      int count = setInMemoryCompactionFlagCounter.incrementAndGet();
3182      if (count <= 1) {
3183        assertTrue(result);
3184      }
3185      if (count == 2) {
3186        assertTrue(!result);
3187      }
3188      return result;
3189    }
3190
3191    @Override
3192    void inMemoryCompaction() {
3193      try {
3194        super.inMemoryCompaction();
3195      } finally {
3196        try {
3197          inMemoryCompactionEndCyclicBarrier.await();
3198        } catch (Throwable e) {
3199          throw new RuntimeException(e);
3200        }
3201
3202      }
3203    }
3204  }
3205
3206  public static class MyCompactingMemStore6 extends CompactingMemStore {
3207    private final CyclicBarrier inMemoryCompactionEndCyclicBarrier = new CyclicBarrier(2);
3208
3209    public MyCompactingMemStore6(Configuration conf, CellComparatorImpl cellComparator,
3210      HStore store, RegionServicesForStores regionServices, MemoryCompactionPolicy compactionPolicy)
3211      throws IOException {
3212      super(conf, cellComparator, store, regionServices, compactionPolicy);
3213    }
3214
3215    @Override
3216    void inMemoryCompaction() {
3217      try {
3218        super.inMemoryCompaction();
3219      } finally {
3220        try {
3221          inMemoryCompactionEndCyclicBarrier.await();
3222        } catch (Throwable e) {
3223          throw new RuntimeException(e);
3224        }
3225
3226      }
3227    }
3228  }
3229
3230  public static class MyDefaultMemStore extends DefaultMemStore {
3231    private static final String GET_SCANNER_THREAD_NAME = "getScannerMyThread";
3232    private static final String FLUSH_THREAD_NAME = "flushMyThread";
3233    /**
3234     * Only when flush thread enters {@link DefaultMemStore#doClearSnapShot}, getScanner thread
3235     * could start.
3236     */
3237    private final CyclicBarrier getScannerCyclicBarrier = new CyclicBarrier(2);
3238    /**
3239     * Used by getScanner thread notifies flush thread {@link DefaultMemStore#getSnapshotSegments}
3240     * completed, {@link DefaultMemStore#doClearSnapShot} could continue.
3241     */
3242    private final CyclicBarrier preClearSnapShotCyclicBarrier = new CyclicBarrier(2);
3243    /**
3244     * Used by flush thread notifies getScanner thread {@link DefaultMemStore#doClearSnapShot}
3245     * completed, {@link DefaultMemStore#getScanners} could continue.
3246     */
3247    private final CyclicBarrier postClearSnapShotCyclicBarrier = new CyclicBarrier(2);
3248    private final AtomicInteger getSnapshotSegmentsCounter = new AtomicInteger(0);
3249    private final AtomicInteger clearSnapshotCounter = new AtomicInteger(0);
3250    private volatile boolean shouldWait = true;
3251    private volatile HStore store = null;
3252
3253    public MyDefaultMemStore(Configuration conf, CellComparator cellComparator,
3254      RegionServicesForStores regionServices) throws IOException {
3255      super(conf, cellComparator, regionServices);
3256    }
3257
3258    @Override
3259    protected List<Segment> getSnapshotSegments() {
3260
3261      List<Segment> result = super.getSnapshotSegments();
3262
3263      if (Thread.currentThread().getName().equals(GET_SCANNER_THREAD_NAME)) {
3264        int currentCount = getSnapshotSegmentsCounter.incrementAndGet();
3265        if (currentCount == 1) {
3266          if (this.shouldWait) {
3267            try {
3268              /**
3269               * Notify flush thread {@link DefaultMemStore#getSnapshotSegments} completed,
3270               * {@link DefaultMemStore#doClearSnapShot} could continue.
3271               */
3272              preClearSnapShotCyclicBarrier.await();
3273              /**
3274               * Wait for {@link DefaultMemStore#doClearSnapShot} completed.
3275               */
3276              postClearSnapShotCyclicBarrier.await();
3277
3278            } catch (Throwable e) {
3279              throw new RuntimeException(e);
3280            }
3281          }
3282        }
3283      }
3284      return result;
3285    }
3286
3287    @Override
3288    protected void doClearSnapShot() {
3289      if (Thread.currentThread().getName().equals(FLUSH_THREAD_NAME)) {
3290        int currentCount = clearSnapshotCounter.incrementAndGet();
3291        if (currentCount == 1) {
3292          try {
3293            if (
3294              ((ReentrantReadWriteLock) store.getStoreEngine().getLock())
3295                .isWriteLockedByCurrentThread()
3296            ) {
3297              shouldWait = false;
3298            }
3299            /**
3300             * Only when flush thread enters {@link DefaultMemStore#doClearSnapShot}, getScanner
3301             * thread could start.
3302             */
3303            getScannerCyclicBarrier.await();
3304
3305            if (shouldWait) {
3306              /**
3307               * Wait for {@link DefaultMemStore#getSnapshotSegments} completed.
3308               */
3309              preClearSnapShotCyclicBarrier.await();
3310            }
3311          } catch (Throwable e) {
3312            throw new RuntimeException(e);
3313          }
3314        }
3315      }
3316      super.doClearSnapShot();
3317
3318      if (Thread.currentThread().getName().equals(FLUSH_THREAD_NAME)) {
3319        int currentCount = clearSnapshotCounter.get();
3320        if (currentCount == 1) {
3321          if (shouldWait) {
3322            try {
3323              /**
3324               * Notify getScanner thread {@link DefaultMemStore#doClearSnapShot} completed,
3325               * {@link DefaultMemStore#getScanners} could continue.
3326               */
3327              postClearSnapShotCyclicBarrier.await();
3328            } catch (Throwable e) {
3329              throw new RuntimeException(e);
3330            }
3331          }
3332        }
3333      }
3334    }
3335  }
3336}