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