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