|
31 | 31 | import static org.mockito.Mockito.verify; |
32 | 32 | import static org.mockito.Mockito.when; |
33 | 33 |
|
| 34 | +import java.io.FileNotFoundException; |
34 | 35 | import java.io.IOException; |
35 | 36 | import java.lang.ref.SoftReference; |
36 | 37 | import java.security.PrivilegedExceptionAction; |
|
44 | 45 | import java.util.NavigableSet; |
45 | 46 | import java.util.Optional; |
46 | 47 | import java.util.TreeSet; |
| 48 | +import java.util.concurrent.BrokenBarrierException; |
47 | 49 | import java.util.concurrent.ConcurrentSkipListSet; |
48 | 50 | import java.util.concurrent.CountDownLatch; |
49 | 51 | import java.util.concurrent.CyclicBarrier; |
@@ -1530,6 +1532,106 @@ public void getScanners(MyStore store) throws IOException { |
1530 | 1532 | } |
1531 | 1533 | } |
1532 | 1534 |
|
| 1535 | + /** |
| 1536 | + * This test is for HBASE-27519, when the {@link StoreScanner} is scanning,the Flush and the |
| 1537 | + * Compaction execute concurrently and theCcompaction compact and archive the flushed |
| 1538 | + * {@link HStoreFile} which is used by {@link StoreScanner#updateReaders}.Before |
| 1539 | + * HBASE-27519,{@link StoreScanner.updateReaders} would throw {@link FileNotFoundException}. |
| 1540 | + */ |
| 1541 | + @Test |
| 1542 | + public void testStoreScannerUpdateReadersWhenFlushAndCompactConcurrently() throws IOException { |
| 1543 | + Configuration conf = HBaseConfiguration.create(); |
| 1544 | + conf.setBoolean(WALFactory.WAL_ENABLED, false); |
| 1545 | + conf.set(DEFAULT_COMPACTION_POLICY_CLASS_KEY, EverythingPolicy.class.getName()); |
| 1546 | + byte[] r0 = Bytes.toBytes("row0"); |
| 1547 | + byte[] r1 = Bytes.toBytes("row1"); |
| 1548 | + final CyclicBarrier cyclicBarrier = new CyclicBarrier(2); |
| 1549 | + final AtomicBoolean shouldWaitRef = new AtomicBoolean(false); |
| 1550 | + // Initialize region |
| 1551 | + final MyStore myStore = initMyStore(name.getMethodName(), conf, new MyStoreHook() { |
| 1552 | + @Override |
| 1553 | + public void getScanners(MyStore store) throws IOException { |
| 1554 | + try { |
| 1555 | + // Here this method is called by StoreScanner.updateReaders which is invoked by the |
| 1556 | + // following TestHStore.flushStore |
| 1557 | + if (shouldWaitRef.get()) { |
| 1558 | + // wait the following compaction Task start |
| 1559 | + cyclicBarrier.await(); |
| 1560 | + // wait the following HStore.closeAndArchiveCompactedFiles end. |
| 1561 | + cyclicBarrier.await(); |
| 1562 | + } |
| 1563 | + } catch (BrokenBarrierException | InterruptedException e) { |
| 1564 | + throw new RuntimeException(e); |
| 1565 | + } |
| 1566 | + } |
| 1567 | + }); |
| 1568 | + |
| 1569 | + final AtomicReference<Throwable> compactionExceptionRef = new AtomicReference<Throwable>(null); |
| 1570 | + Runnable compactionTask = () -> { |
| 1571 | + try { |
| 1572 | + // Only when the StoreScanner.updateReaders invoked by TestHStore.flushStore prepares for |
| 1573 | + // entering the MyStore.getScanners, compactionTask could start. |
| 1574 | + cyclicBarrier.await(); |
| 1575 | + region.compactStore(family, new NoLimitThroughputController()); |
| 1576 | + myStore.closeAndArchiveCompactedFiles(); |
| 1577 | + // Notify StoreScanner.updateReaders could enter MyStore.getScanners. |
| 1578 | + cyclicBarrier.await(); |
| 1579 | + } catch (Throwable e) { |
| 1580 | + compactionExceptionRef.set(e); |
| 1581 | + } |
| 1582 | + }; |
| 1583 | + |
| 1584 | + long ts = EnvironmentEdgeManager.currentTime(); |
| 1585 | + long seqId = 100; |
| 1586 | + byte[] value = Bytes.toBytes("value"); |
| 1587 | + // older data whihc shouldn't be "seen" by client |
| 1588 | + myStore.add(createCell(r0, qf1, ts, seqId, value), null); |
| 1589 | + flushStore(myStore, id++); |
| 1590 | + myStore.add(createCell(r0, qf2, ts, seqId, value), null); |
| 1591 | + flushStore(myStore, id++); |
| 1592 | + myStore.add(createCell(r0, qf3, ts, seqId, value), null); |
| 1593 | + TreeSet<byte[]> quals = new TreeSet<>(Bytes.BYTES_COMPARATOR); |
| 1594 | + quals.add(qf1); |
| 1595 | + quals.add(qf2); |
| 1596 | + quals.add(qf3); |
| 1597 | + |
| 1598 | + myStore.add(createCell(r1, qf1, ts, seqId, value), null); |
| 1599 | + myStore.add(createCell(r1, qf2, ts, seqId, value), null); |
| 1600 | + myStore.add(createCell(r1, qf3, ts, seqId, value), null); |
| 1601 | + |
| 1602 | + Thread.currentThread() |
| 1603 | + .setName("testStoreScannerUpdateReadersWhenFlushAndCompactConcurrently thread"); |
| 1604 | + Scan scan = new Scan(); |
| 1605 | + scan.withStartRow(r0, true); |
| 1606 | + try (InternalScanner scanner = (InternalScanner) myStore.getScanner(scan, quals, seqId)) { |
| 1607 | + List<Cell> results = new MyList<>(size -> { |
| 1608 | + switch (size) { |
| 1609 | + case 1: |
| 1610 | + shouldWaitRef.set(true); |
| 1611 | + Thread thread = new Thread(compactionTask); |
| 1612 | + thread.setName("MyCompacting Thread."); |
| 1613 | + thread.start(); |
| 1614 | + try { |
| 1615 | + flushStore(myStore, id++); |
| 1616 | + thread.join(); |
| 1617 | + } catch (IOException | InterruptedException e) { |
| 1618 | + throw new RuntimeException(e); |
| 1619 | + } |
| 1620 | + shouldWaitRef.set(false); |
| 1621 | + break; |
| 1622 | + default: |
| 1623 | + break; |
| 1624 | + } |
| 1625 | + }); |
| 1626 | + // Before HBASE-27519, here would throw java.io.FileNotFoundException because the storeFile |
| 1627 | + // which used by StoreScanner.updateReaders is deleted by compactionTask. |
| 1628 | + scanner.next(results); |
| 1629 | + // The results is r0 row cells. |
| 1630 | + assertEquals(3, results.size()); |
| 1631 | + assertTrue(compactionExceptionRef.get() == null); |
| 1632 | + } |
| 1633 | + } |
| 1634 | + |
1533 | 1635 | @Test |
1534 | 1636 | public void testReclaimChunkWhenScaning() throws IOException { |
1535 | 1637 | init("testReclaimChunkWhenScaning"); |
|
0 commit comments