Skip to content

Commit e65fc92

Browse files
authored
HBASE-26069 Remove HStore.compactRecentForTestingAssumingDefaultPolic… (#3462)
Signed-off-by: Yulin Niu <[email protected]>
1 parent 29cd782 commit e65fc92

File tree

5 files changed

+121
-184
lines changed

5 files changed

+121
-184
lines changed

hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java

Lines changed: 0 additions & 61 deletions
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,4 @@
11
/*
2-
*
32
* Licensed to the Apache Software Foundation (ASF) under one
43
* or more contributor license agreements. See the NOTICE file
54
* distributed with this work for additional information
@@ -53,7 +52,6 @@
5352
import java.util.function.ToLongFunction;
5453
import java.util.stream.Collectors;
5554
import java.util.stream.LongStream;
56-
5755
import org.apache.hadoop.conf.Configuration;
5856
import org.apache.hadoop.fs.FileSystem;
5957
import org.apache.hadoop.fs.Path;
@@ -89,7 +87,6 @@
8987
import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
9088
import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
9189
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl;
92-
import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
9390
import org.apache.hadoop.hbase.regionserver.compactions.OffPeakHours;
9491
import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
9592
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
@@ -1747,64 +1744,6 @@ public void replayCompactionMarker(CompactionDescriptor compaction, boolean pick
17471744
}
17481745
}
17491746

1750-
/**
1751-
* This method tries to compact N recent files for testing.
1752-
* Note that because compacting "recent" files only makes sense for some policies,
1753-
* e.g. the default one, it assumes default policy is used. It doesn't use policy,
1754-
* but instead makes a compaction candidate list by itself.
1755-
* @param N Number of files.
1756-
*/
1757-
public void compactRecentForTestingAssumingDefaultPolicy(int N) throws IOException {
1758-
List<HStoreFile> filesToCompact;
1759-
boolean isMajor;
1760-
1761-
this.lock.readLock().lock();
1762-
try {
1763-
synchronized (filesCompacting) {
1764-
filesToCompact = Lists.newArrayList(storeEngine.getStoreFileManager().getStorefiles());
1765-
if (!filesCompacting.isEmpty()) {
1766-
// exclude all files older than the newest file we're currently
1767-
// compacting. this allows us to preserve contiguity (HBASE-2856)
1768-
HStoreFile last = filesCompacting.get(filesCompacting.size() - 1);
1769-
int idx = filesToCompact.indexOf(last);
1770-
Preconditions.checkArgument(idx != -1);
1771-
filesToCompact.subList(0, idx + 1).clear();
1772-
}
1773-
int count = filesToCompact.size();
1774-
if (N > count) {
1775-
throw new RuntimeException("Not enough files");
1776-
}
1777-
1778-
filesToCompact = filesToCompact.subList(count - N, count);
1779-
isMajor = (filesToCompact.size() == storeEngine.getStoreFileManager().getStorefileCount());
1780-
filesCompacting.addAll(filesToCompact);
1781-
Collections.sort(filesCompacting, storeEngine.getStoreFileManager()
1782-
.getStoreFileComparator());
1783-
}
1784-
} finally {
1785-
this.lock.readLock().unlock();
1786-
}
1787-
1788-
try {
1789-
// Ready to go. Have list of files to compact.
1790-
List<Path> newFiles = ((DefaultCompactor)this.storeEngine.getCompactor())
1791-
.compactForTesting(filesToCompact, isMajor);
1792-
for (Path newFile: newFiles) {
1793-
// Move the compaction into place.
1794-
HStoreFile sf = moveFileIntoPlace(newFile);
1795-
if (this.getCoprocessorHost() != null) {
1796-
this.getCoprocessorHost().postCompact(this, sf, null, null, null);
1797-
}
1798-
replaceStoreFiles(filesToCompact, Collections.singletonList(sf));
1799-
refreshStoreSizeAndTotalBytes();
1800-
}
1801-
} finally {
1802-
synchronized (filesCompacting) {
1803-
filesCompacting.removeAll(filesToCompact);
1804-
}
1805-
}
1806-
}
1807-
18081747
@Override
18091748
public boolean hasReferences() {
18101749
// Grab the read lock here, because we need to ensure that: only when the atomic

hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/DefaultCompactor.java

Lines changed: 1 addition & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -18,21 +18,18 @@
1818
package org.apache.hadoop.hbase.regionserver.compactions;
1919

2020
import java.io.IOException;
21-
import java.util.Collection;
2221
import java.util.List;
23-
2422
import org.apache.hadoop.conf.Configuration;
2523
import org.apache.hadoop.fs.Path;
2624
import org.apache.hadoop.hbase.regionserver.HStore;
27-
import org.apache.hadoop.hbase.regionserver.HStoreFile;
2825
import org.apache.hadoop.hbase.regionserver.InternalScanner;
2926
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
30-
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
3127
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
3228
import org.apache.hadoop.hbase.security.User;
3329
import org.apache.yetus.audience.InterfaceAudience;
3430
import org.slf4j.Logger;
3531
import org.slf4j.LoggerFactory;
32+
3633
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
3734

3835
/**
@@ -65,23 +62,6 @@ public List<Path> compact(final CompactionRequestImpl request,
6562
return compact(request, defaultScannerFactory, writerFactory, throughputController, user);
6663
}
6764

68-
/**
69-
* Compact a list of files for testing. Creates a fake {@link CompactionRequestImpl} to pass to
70-
* {@link #compact(CompactionRequestImpl, ThroughputController, User)};
71-
* @param filesToCompact the files to compact. These are used as the compactionSelection for the
72-
* generated {@link CompactionRequestImpl}.
73-
* @param isMajor true to major compact (prune all deletes, max versions, etc)
74-
* @return Product of compaction or an empty list if all cells expired or deleted and nothing \
75-
* made it through the compaction.
76-
* @throws IOException
77-
*/
78-
public List<Path> compactForTesting(Collection<HStoreFile> filesToCompact, boolean isMajor)
79-
throws IOException {
80-
CompactionRequestImpl cr = new CompactionRequestImpl(filesToCompact);
81-
cr.setIsMajor(isMajor, isMajor);
82-
return compact(cr, NoLimitThroughputController.INSTANCE, null);
83-
}
84-
8565
@Override
8666
protected List<Path> commitWriter(StoreFileWriter writer, FileDetails fd,
8767
CompactionRequestImpl request) throws IOException {

hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestScannerSelectionUsingTTL.java

Lines changed: 3 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -37,7 +37,6 @@
3737
import org.apache.hadoop.hbase.client.TableDescriptor;
3838
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
3939
import org.apache.hadoop.hbase.regionserver.HRegion;
40-
import org.apache.hadoop.hbase.regionserver.HStore;
4140
import org.apache.hadoop.hbase.regionserver.InternalScanner;
4241
import org.apache.hadoop.hbase.testclassification.IOTests;
4342
import org.apache.hadoop.hbase.testclassification.LargeTests;
@@ -82,25 +81,18 @@ public class TestScannerSelectionUsingTTL {
8281

8382
public final int numFreshFiles, totalNumFiles;
8483

85-
/** Whether we are specifying the exact files to compact */
86-
private final boolean explicitCompaction;
87-
8884
@Parameters
8985
public static Collection<Object[]> parameters() {
9086
List<Object[]> params = new ArrayList<>();
9187
for (int numFreshFiles = 1; numFreshFiles <= 3; ++numFreshFiles) {
92-
for (boolean explicitCompaction : new boolean[] { false, true }) {
93-
params.add(new Object[] { numFreshFiles, explicitCompaction });
94-
}
88+
params.add(new Object[] { numFreshFiles });
9589
}
9690
return params;
9791
}
9892

99-
public TestScannerSelectionUsingTTL(int numFreshFiles,
100-
boolean explicitCompaction) {
93+
public TestScannerSelectionUsingTTL(int numFreshFiles) {
10194
this.numFreshFiles = numFreshFiles;
10295
this.totalNumFiles = numFreshFiles + NUM_EXPIRED_FILES;
103-
this.explicitCompaction = explicitCompaction;
10496
}
10597

10698
@Test
@@ -152,13 +144,7 @@ public void testScannerSelection() throws IOException {
152144
Set<String> accessedFiles = cache.getCachedFileNamesForTest();
153145
LOG.debug("Files accessed during scan: " + accessedFiles);
154146

155-
// Exercise both compaction codepaths.
156-
if (explicitCompaction) {
157-
HStore store = region.getStore(FAMILY_BYTES);
158-
store.compactRecentForTestingAssumingDefaultPolicy(totalNumFiles);
159-
} else {
160-
region.compact(false);
161-
}
147+
region.compact(false);
162148

163149
HBaseTestingUtility.closeRegionAndWAL(region);
164150
}

hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompaction.java

Lines changed: 33 additions & 30 deletions
Original file line numberDiff line numberDiff line change
@@ -85,6 +85,8 @@
8585
import org.mockito.Mockito;
8686
import org.mockito.invocation.InvocationOnMock;
8787
import org.mockito.stubbing.Answer;
88+
import org.slf4j.Logger;
89+
import org.slf4j.LoggerFactory;
8890

8991
/**
9092
* Test compaction framework and common functions
@@ -94,9 +96,12 @@ public class TestCompaction {
9496

9597
@ClassRule
9698
public static final HBaseClassTestRule CLASS_RULE =
97-
HBaseClassTestRule.forClass(TestCompaction.class);
99+
HBaseClassTestRule.forClass(TestCompaction.class);
98100

99-
@Rule public TestName name = new TestName();
101+
private static final Logger LOG = LoggerFactory.getLogger(TestCompaction.class);
102+
103+
@Rule
104+
public TestName name = new TestName();
100105
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
101106
protected Configuration conf = UTIL.getConfiguration();
102107

@@ -154,7 +159,6 @@ public void tearDown() throws Exception {
154159
/**
155160
* Verify that you can stop a long-running compaction
156161
* (used during RS shutdown)
157-
* @throws Exception
158162
*/
159163
@Test
160164
public void testInterruptCompactionBySize() throws Exception {
@@ -180,7 +184,7 @@ public void testInterruptCompactionBySize() throws Exception {
180184
}
181185

182186
HRegion spyR = spy(r);
183-
doAnswer(new Answer() {
187+
doAnswer(new Answer<Object>() {
184188
@Override
185189
public Object answer(InvocationOnMock invocation) throws Throwable {
186190
r.writestate.writesEnabled = false;
@@ -256,7 +260,7 @@ public void testInterruptCompactionByTime() throws Exception {
256260
}
257261

258262
HRegion spyR = spy(r);
259-
doAnswer(new Answer() {
263+
doAnswer(new Answer<Object>() {
260264
@Override
261265
public Object answer(InvocationOnMock invocation) throws Throwable {
262266
r.writestate.writesEnabled = false;
@@ -311,15 +315,14 @@ public Object answer(InvocationOnMock invocation) throws Throwable {
311315

312316
private int count() throws IOException {
313317
int count = 0;
314-
for (HStoreFile f: this.r.stores.
315-
get(COLUMN_FAMILY_TEXT).getStorefiles()) {
318+
for (HStoreFile f : this.r.stores.get(COLUMN_FAMILY_TEXT).getStorefiles()) {
316319
HFileScanner scanner = f.getReader().getScanner(false, false);
317320
if (!scanner.seekTo()) {
318321
continue;
319322
}
320323
do {
321324
count++;
322-
} while(scanner.next());
325+
} while (scanner.next());
323326
}
324327
return count;
325328
}
@@ -344,7 +347,8 @@ public void testCompactionWithCorruptResult() throws Exception {
344347

345348
Collection<HStoreFile> storeFiles = store.getStorefiles();
346349
DefaultCompactor tool = (DefaultCompactor)store.storeEngine.getCompactor();
347-
tool.compactForTesting(storeFiles, false);
350+
CompactionRequestImpl request = new CompactionRequestImpl(storeFiles);
351+
tool.compact(request, NoLimitThroughputController.INSTANCE, null);
348352

349353
// Now lets corrupt the compacted file.
350354
FileSystem fs = store.getFileSystem();
@@ -363,7 +367,7 @@ public void testCompactionWithCorruptResult() throws Exception {
363367
// in the 'tmp' directory;
364368
assertTrue(fs.exists(origPath));
365369
assertFalse(fs.exists(dstPath));
366-
System.out.println("testCompactionWithCorruptResult Passed");
370+
LOG.info("testCompactionWithCorruptResult Passed");
367371
return;
368372
}
369373
fail("testCompactionWithCorruptResult failed since no exception was" +
@@ -418,28 +422,27 @@ public void testCompactionFailure() throws Exception {
418422
Mockito.when(mockRegion.checkSplit()).
419423
thenThrow(new RuntimeException("Thrown intentionally by test!"));
420424

421-
MetricsRegionWrapper metricsWrapper = new MetricsRegionWrapperImpl(r);
425+
try (MetricsRegionWrapperImpl metricsWrapper = new MetricsRegionWrapperImpl(r)) {
422426

423-
long preCompletedCount = metricsWrapper.getNumCompactionsCompleted();
424-
long preFailedCount = metricsWrapper.getNumCompactionsFailed();
427+
long preCompletedCount = metricsWrapper.getNumCompactionsCompleted();
428+
long preFailedCount = metricsWrapper.getNumCompactionsFailed();
425429

426-
CountDownLatch latch = new CountDownLatch(1);
427-
Tracker tracker = new Tracker(latch);
428-
thread.requestCompaction(mockRegion, store, "test custom comapction", PRIORITY_USER,
429-
tracker, null);
430-
// wait for the latch to complete.
431-
latch.await(120, TimeUnit.SECONDS);
432-
433-
// compaction should have completed and been marked as failed due to error in split request
434-
long postCompletedCount = metricsWrapper.getNumCompactionsCompleted();
435-
long postFailedCount = metricsWrapper.getNumCompactionsFailed();
436-
437-
assertTrue("Completed count should have increased (pre=" + preCompletedCount +
438-
", post="+postCompletedCount+")",
439-
postCompletedCount > preCompletedCount);
440-
assertTrue("Failed count should have increased (pre=" + preFailedCount +
441-
", post=" + postFailedCount + ")",
442-
postFailedCount > preFailedCount);
430+
CountDownLatch latch = new CountDownLatch(1);
431+
Tracker tracker = new Tracker(latch);
432+
thread.requestCompaction(mockRegion, store, "test custom comapction", PRIORITY_USER, tracker,
433+
null);
434+
// wait for the latch to complete.
435+
latch.await(120, TimeUnit.SECONDS);
436+
437+
// compaction should have completed and been marked as failed due to error in split request
438+
long postCompletedCount = metricsWrapper.getNumCompactionsCompleted();
439+
long postFailedCount = metricsWrapper.getNumCompactionsFailed();
440+
441+
assertTrue("Completed count should have increased (pre=" + preCompletedCount + ", post=" +
442+
postCompletedCount + ")", postCompletedCount > preCompletedCount);
443+
assertTrue("Failed count should have increased (pre=" + preFailedCount + ", post=" +
444+
postFailedCount + ")", postFailedCount > preFailedCount);
445+
}
443446
}
444447

445448
/**

0 commit comments

Comments
 (0)