Skip to content

Commit b86d2d4

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

File tree

5 files changed

+125
-190
lines changed

5 files changed

+125
-190
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;
@@ -1761,64 +1758,6 @@ public void replayCompactionMarker(CompactionDescriptor compaction, boolean pick
17611758
}
17621759
}
17631760

1764-
/**
1765-
* This method tries to compact N recent files for testing.
1766-
* Note that because compacting "recent" files only makes sense for some policies,
1767-
* e.g. the default one, it assumes default policy is used. It doesn't use policy,
1768-
* but instead makes a compaction candidate list by itself.
1769-
* @param N Number of files.
1770-
*/
1771-
public void compactRecentForTestingAssumingDefaultPolicy(int N) throws IOException {
1772-
List<HStoreFile> filesToCompact;
1773-
boolean isMajor;
1774-
1775-
this.lock.readLock().lock();
1776-
try {
1777-
synchronized (filesCompacting) {
1778-
filesToCompact = Lists.newArrayList(storeEngine.getStoreFileManager().getStorefiles());
1779-
if (!filesCompacting.isEmpty()) {
1780-
// exclude all files older than the newest file we're currently
1781-
// compacting. this allows us to preserve contiguity (HBASE-2856)
1782-
HStoreFile last = filesCompacting.get(filesCompacting.size() - 1);
1783-
int idx = filesToCompact.indexOf(last);
1784-
Preconditions.checkArgument(idx != -1);
1785-
filesToCompact.subList(0, idx + 1).clear();
1786-
}
1787-
int count = filesToCompact.size();
1788-
if (N > count) {
1789-
throw new RuntimeException("Not enough files");
1790-
}
1791-
1792-
filesToCompact = filesToCompact.subList(count - N, count);
1793-
isMajor = (filesToCompact.size() == storeEngine.getStoreFileManager().getStorefileCount());
1794-
filesCompacting.addAll(filesToCompact);
1795-
Collections.sort(filesCompacting, storeEngine.getStoreFileManager()
1796-
.getStoreFileComparator());
1797-
}
1798-
} finally {
1799-
this.lock.readLock().unlock();
1800-
}
1801-
1802-
try {
1803-
// Ready to go. Have list of files to compact.
1804-
List<Path> newFiles = ((DefaultCompactor)this.storeEngine.getCompactor())
1805-
.compactForTesting(filesToCompact, isMajor);
1806-
for (Path newFile: newFiles) {
1807-
// Move the compaction into place.
1808-
HStoreFile sf = moveFileIntoPlace(newFile);
1809-
if (this.getCoprocessorHost() != null) {
1810-
this.getCoprocessorHost().postCompact(this, sf, null, null, null);
1811-
}
1812-
replaceStoreFiles(filesToCompact, Collections.singletonList(sf));
1813-
refreshStoreSizeAndTotalBytes();
1814-
}
1815-
} finally {
1816-
synchronized (filesCompacting) {
1817-
filesCompacting.removeAll(filesToCompact);
1818-
}
1819-
}
1820-
}
1821-
18221761
@Override
18231762
public boolean hasReferences() {
18241763
// 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
@@ -81,6 +81,8 @@
8181
import org.mockito.Mockito;
8282
import org.mockito.invocation.InvocationOnMock;
8383
import org.mockito.stubbing.Answer;
84+
import org.slf4j.Logger;
85+
import org.slf4j.LoggerFactory;
8486

8587
/**
8688
* Test compaction framework and common functions
@@ -90,10 +92,13 @@ public class TestCompaction {
9092

9193
@ClassRule
9294
public static final HBaseClassTestRule CLASS_RULE =
93-
HBaseClassTestRule.forClass(TestCompaction.class);
95+
HBaseClassTestRule.forClass(TestCompaction.class);
9496

95-
@Rule public TestName name = new TestName();
96-
private static final HBaseTestingUtility UTIL = HBaseTestingUtility.createLocalHTU();
97+
private static final Logger LOG = LoggerFactory.getLogger(TestCompaction.class);
98+
99+
@Rule
100+
public TestName name = new TestName();
101+
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
97102
protected Configuration conf = UTIL.getConfiguration();
98103

99104
private HRegion r = null;
@@ -150,7 +155,6 @@ public void tearDown() throws Exception {
150155
/**
151156
* Verify that you can stop a long-running compaction
152157
* (used during RS shutdown)
153-
* @throws Exception
154158
*/
155159
@Test
156160
public void testInterruptCompaction() throws Exception {
@@ -177,7 +181,7 @@ public void testInterruptCompaction() throws Exception {
177181
}
178182

179183
HRegion spyR = spy(r);
180-
doAnswer(new Answer() {
184+
doAnswer(new Answer<Object>() {
181185
@Override
182186
public Object answer(InvocationOnMock invocation) throws Throwable {
183187
r.writestate.writesEnabled = false;
@@ -231,15 +235,14 @@ public Object answer(InvocationOnMock invocation) throws Throwable {
231235

232236
private int count() throws IOException {
233237
int count = 0;
234-
for (HStoreFile f: this.r.stores.
235-
get(COLUMN_FAMILY_TEXT).getStorefiles()) {
238+
for (HStoreFile f : this.r.stores.get(COLUMN_FAMILY_TEXT).getStorefiles()) {
236239
HFileScanner scanner = f.getReader().getScanner(false, false);
237240
if (!scanner.seekTo()) {
238241
continue;
239242
}
240243
do {
241244
count++;
242-
} while(scanner.next());
245+
} while (scanner.next());
243246
}
244247
return count;
245248
}
@@ -264,7 +267,8 @@ public void testCompactionWithCorruptResult() throws Exception {
264267

265268
Collection<HStoreFile> storeFiles = store.getStorefiles();
266269
DefaultCompactor tool = (DefaultCompactor)store.storeEngine.getCompactor();
267-
tool.compactForTesting(storeFiles, false);
270+
CompactionRequestImpl request = new CompactionRequestImpl(storeFiles);
271+
tool.compact(request, NoLimitThroughputController.INSTANCE, null);
268272

269273
// Now lets corrupt the compacted file.
270274
FileSystem fs = store.getFileSystem();
@@ -283,7 +287,7 @@ public void testCompactionWithCorruptResult() throws Exception {
283287
// in the 'tmp' directory;
284288
assertTrue(fs.exists(origPath));
285289
assertFalse(fs.exists(dstPath));
286-
System.out.println("testCompactionWithCorruptResult Passed");
290+
LOG.info("testCompactionWithCorruptResult Passed");
287291
return;
288292
}
289293
fail("testCompactionWithCorruptResult failed since no exception was" +
@@ -338,28 +342,27 @@ public void testCompactionFailure() throws Exception {
338342
Mockito.when(mockRegion.checkSplit()).
339343
thenThrow(new RuntimeException("Thrown intentionally by test!"));
340344

341-
MetricsRegionWrapper metricsWrapper = new MetricsRegionWrapperImpl(r);
345+
try (MetricsRegionWrapperImpl metricsWrapper = new MetricsRegionWrapperImpl(r)) {
342346

343-
long preCompletedCount = metricsWrapper.getNumCompactionsCompleted();
344-
long preFailedCount = metricsWrapper.getNumCompactionsFailed();
347+
long preCompletedCount = metricsWrapper.getNumCompactionsCompleted();
348+
long preFailedCount = metricsWrapper.getNumCompactionsFailed();
345349

346-
CountDownLatch latch = new CountDownLatch(1);
347-
Tracker tracker = new Tracker(latch);
348-
thread.requestCompaction(mockRegion, store, "test custom comapction", PRIORITY_USER,
349-
tracker, null);
350-
// wait for the latch to complete.
351-
latch.await(120, TimeUnit.SECONDS);
352-
353-
// compaction should have completed and been marked as failed due to error in split request
354-
long postCompletedCount = metricsWrapper.getNumCompactionsCompleted();
355-
long postFailedCount = metricsWrapper.getNumCompactionsFailed();
356-
357-
assertTrue("Completed count should have increased (pre=" + preCompletedCount +
358-
", post="+postCompletedCount+")",
359-
postCompletedCount > preCompletedCount);
360-
assertTrue("Failed count should have increased (pre=" + preFailedCount +
361-
", post=" + postFailedCount + ")",
362-
postFailedCount > preFailedCount);
350+
CountDownLatch latch = new CountDownLatch(1);
351+
Tracker tracker = new Tracker(latch);
352+
thread.requestCompaction(mockRegion, store, "test custom comapction", PRIORITY_USER, tracker,
353+
null);
354+
// wait for the latch to complete.
355+
latch.await(120, TimeUnit.SECONDS);
356+
357+
// compaction should have completed and been marked as failed due to error in split request
358+
long postCompletedCount = metricsWrapper.getNumCompactionsCompleted();
359+
long postFailedCount = metricsWrapper.getNumCompactionsFailed();
360+
361+
assertTrue("Completed count should have increased (pre=" + preCompletedCount + ", post=" +
362+
postCompletedCount + ")", postCompletedCount > preCompletedCount);
363+
assertTrue("Failed count should have increased (pre=" + preFailedCount + ", post=" +
364+
postFailedCount + ")", postFailedCount > preFailedCount);
365+
}
363366
}
364367

365368
/**

0 commit comments

Comments
 (0)