Skip to content

Commit 29cd0bf

Browse files
committed
HBASE-26229 Limit count and size of L0 files compaction in StripeCompactionPolicy
1 parent 889049e commit 29cd0bf

File tree

3 files changed

+107
-23
lines changed

3 files changed

+107
-23
lines changed

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

Lines changed: 20 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,6 @@
2222
import java.io.IOException;
2323
import java.util.ArrayList;
2424
import java.util.List;
25-
2625
import org.apache.hadoop.conf.Configuration;
2726
import org.apache.hadoop.hbase.regionserver.HStoreFile;
2827
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
@@ -120,10 +119,29 @@ public List<HStoreFile> applyCompactionPolicy(List<HStoreFile> candidates, boole
120119
}
121120
LOG.debug("Exploring compaction algorithm has selected {} files of size {} starting at " +
122121
"candidate #{} after considering {} permutations with {} in ratio", bestSelection.size(),
123-
bestSize, bestSize, opts, optsInRatio);
122+
bestSize, bestStart, opts, optsInRatio);
124123
return new ArrayList<>(bestSelection);
125124
}
126125

126+
/**
127+
* Select at least one file in the candidates list to compact, through choosing files
128+
* from the head to the index that the accumulation length larger the max compaction size.
129+
* This method is a supplementary of the selectSimpleCompaction() method, aims to make sure
130+
* at least one file can be selected to compact, for compactions like L0 files, which need to
131+
* compact all files and as soon as possible.
132+
*/
133+
public List<HStoreFile> selectCompactFiles(final List<HStoreFile> candidates, int maxFiles,
134+
boolean isOffpeak) {
135+
long selectedSize = 0L;
136+
for (int end = 0; end < Math.min(candidates.size(), maxFiles); end++) {
137+
selectedSize += candidates.get(end).getReader().length();
138+
if (selectedSize >= comConf.getMaxCompactSize(isOffpeak)) {
139+
return candidates.subList(0, end + 1);
140+
}
141+
}
142+
return candidates;
143+
}
144+
127145
private boolean isBetterSelection(List<HStoreFile> bestSelection, long bestSize,
128146
List<HStoreFile> selection, long size, boolean mightBeStuck) {
129147
if (mightBeStuck && bestSize > 0 && size > 0) {

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

Lines changed: 49 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,6 @@
1919
package org.apache.hadoop.hbase.regionserver.compactions;
2020

2121
import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_KEY;
22-
2322
import java.io.IOException;
2423
import java.util.ArrayList;
2524
import java.util.Collection;
@@ -130,27 +129,33 @@ public StripeCompactionRequest selectCompaction(StripeInformationProvider si,
130129
List<HStoreFile> l0Files = si.getLevel0Files();
131130

132131
// See if we need to make new stripes.
133-
boolean shouldCompactL0 = (this.config.getLevel0MinFiles() <= l0Files.size());
132+
boolean shouldCompactL0 = this.config.getLevel0MinFiles() <= l0Files.size();
134133
if (stripeCount == 0) {
135-
if (!shouldCompactL0) return null; // nothing to do.
136-
return selectNewStripesCompaction(si);
134+
if (!shouldCompactL0) {
135+
return null; // nothing to do.
136+
}
137+
return selectL0OnlyCompaction(si);
137138
}
138139

139140
boolean canDropDeletesNoL0 = l0Files.isEmpty();
140141
if (shouldCompactL0) {
141142
if (!canDropDeletesNoL0) {
142143
// If we need to compact L0, see if we can add something to it, and drop deletes.
143-
StripeCompactionRequest result = selectSingleStripeCompaction(
144-
si, true, canDropDeletesNoL0, isOffpeak);
145-
if (result != null) return result;
144+
StripeCompactionRequest result = selectSingleStripeCompaction(si, !shouldSelectL0Files(si),
145+
canDropDeletesNoL0, isOffpeak);
146+
if (result != null) {
147+
return result;
148+
}
146149
}
147150
LOG.debug("Selecting L0 compaction with " + l0Files.size() + " files");
148-
return new BoundaryStripeCompactionRequest(l0Files, si.getStripeBoundaries());
151+
return selectL0OnlyCompaction(si);
149152
}
150153

151154
// Try to delete fully expired stripes
152155
StripeCompactionRequest result = selectExpiredMergeCompaction(si, canDropDeletesNoL0);
153-
if (result != null) return result;
156+
if (result != null) {
157+
return result;
158+
}
154159

155160
// Ok, nothing special here, let's see if we need to do a common compaction.
156161
// This will also split the stripes that are too big if needed.
@@ -200,7 +205,7 @@ protected StripeCompactionRequest selectSingleStripeCompaction(StripeInformation
200205
// If we want to compact L0 to drop deletes, we only want whole-stripe compactions.
201206
// So, pass includeL0 as 2nd parameter to indicate that.
202207
List<HStoreFile> selection = selectSimpleCompaction(stripes.get(i),
203-
!canDropDeletesWithoutL0 && includeL0, isOffpeak);
208+
!canDropDeletesWithoutL0 && includeL0, isOffpeak, false);
204209
if (selection.isEmpty()) continue;
205210
long size = 0;
206211
for (HStoreFile sf : selection) {
@@ -268,21 +273,46 @@ protected StripeCompactionRequest selectSingleStripeCompaction(StripeInformation
268273
* @return The resulting selection.
269274
*/
270275
private List<HStoreFile> selectSimpleCompaction(
271-
List<HStoreFile> sfs, boolean allFilesOnly, boolean isOffpeak) {
276+
List<HStoreFile> sfs, boolean allFilesOnly, boolean isOffpeak, boolean forceCompact) {
272277
int minFilesLocal = Math.max(
273278
allFilesOnly ? sfs.size() : 0, this.config.getStripeCompactMinFiles());
274279
int maxFilesLocal = Math.max(this.config.getStripeCompactMaxFiles(), minFilesLocal);
275-
return stripePolicy.applyCompactionPolicy(sfs, false, isOffpeak, minFilesLocal, maxFilesLocal);
280+
List<HStoreFile> selected = stripePolicy.applyCompactionPolicy(sfs, false,
281+
isOffpeak, minFilesLocal, maxFilesLocal);
282+
if (forceCompact && (selected == null || selected.isEmpty()) && !sfs.isEmpty()) {
283+
return stripePolicy.selectCompactFiles(sfs, maxFilesLocal, isOffpeak);
284+
}
285+
return selected;
286+
}
287+
288+
private boolean shouldSelectL0Files(StripeInformationProvider si) {
289+
return si.getLevel0Files().size() > this.config.getStripeCompactMaxFiles() ||
290+
getTotalFileSize(si.getLevel0Files()) > comConf.getMaxCompactSize();
276291
}
277292

278-
private StripeCompactionRequest selectNewStripesCompaction(StripeInformationProvider si) {
293+
private StripeCompactionRequest selectL0OnlyCompaction(StripeInformationProvider si) {
279294
List<HStoreFile> l0Files = si.getLevel0Files();
280-
Pair<Long, Integer> kvsAndCount = estimateTargetKvs(l0Files, config.getInitialCount());
281-
LOG.debug("Creating " + kvsAndCount.getSecond() + " initial stripes with "
282-
+ kvsAndCount.getFirst() + " kvs each via L0 compaction of " + l0Files.size() + " files");
283-
SplitStripeCompactionRequest request = new SplitStripeCompactionRequest(
284-
si.getLevel0Files(), OPEN_KEY, OPEN_KEY, kvsAndCount.getSecond(), kvsAndCount.getFirst());
285-
request.setMajorRangeFull(); // L0 only, can drop deletes.
295+
List<HStoreFile> selectedFiles = l0Files;
296+
if (shouldSelectL0Files(si)) {
297+
selectedFiles = selectSimpleCompaction(l0Files, false, false, true);
298+
assert !selectedFiles.isEmpty() : "Selected L0 files should not be empty";
299+
}
300+
StripeCompactionRequest request;
301+
if (si.getStripeCount() == 0) {
302+
Pair<Long, Integer> estimate = estimateTargetKvs(selectedFiles, config.getInitialCount());
303+
long targetKvs = estimate.getFirst();
304+
int targetCount = estimate.getSecond();
305+
request =
306+
new SplitStripeCompactionRequest(selectedFiles, OPEN_KEY, OPEN_KEY, targetCount, targetKvs);
307+
if (selectedFiles.size() == l0Files.size()) {
308+
((SplitStripeCompactionRequest) request).setMajorRangeFull(); // L0 only, can drop deletes.
309+
}
310+
LOG.debug("Creating {} initial stripes with {} kvs each via L0 compaction of {}/{} files",
311+
targetCount, targetKvs, selectedFiles.size(), l0Files.size());
312+
} else {
313+
request = new BoundaryStripeCompactionRequest(selectedFiles, si.getStripeBoundaries());
314+
LOG.debug("Boundary L0 compaction of {}/{} files", selectedFiles.size(), l0Files.size());
315+
}
286316
return request;
287317
}
288318

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

Lines changed: 38 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,9 +17,12 @@
1717
*/
1818
package org.apache.hadoop.hbase.regionserver.compactions;
1919

20+
import static org.apache.hadoop.hbase.regionserver.StripeStoreConfig.MAX_FILES_KEY;
2021
import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_KEY;
22+
import static org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration.HBASE_HSTORE_COMPACTION_MAX_SIZE_KEY;
2123
import static org.junit.Assert.assertEquals;
2224
import static org.junit.Assert.assertFalse;
25+
import static org.junit.Assert.assertNotNull;
2326
import static org.junit.Assert.assertNull;
2427
import static org.junit.Assert.assertTrue;
2528
import static org.mockito.AdditionalMatchers.aryEq;
@@ -36,7 +39,6 @@
3639
import static org.mockito.Mockito.times;
3740
import static org.mockito.Mockito.verify;
3841
import static org.mockito.Mockito.when;
39-
4042
import java.io.IOException;
4143
import java.util.ArrayList;
4244
import java.util.Arrays;
@@ -88,7 +90,6 @@
8890
import org.junit.runners.Parameterized.Parameter;
8991
import org.junit.runners.Parameterized.Parameters;
9092
import org.mockito.ArgumentMatcher;
91-
9293
import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
9394
import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
9495

@@ -268,6 +269,41 @@ public void testInitialCountFromL0() throws Exception {
268269
verifyCompaction(policy, si, si.getStorefiles(), true, 6, 5L, OPEN_KEY, OPEN_KEY, true);
269270
}
270271

272+
@Test
273+
public void testSelectL0Compaction() throws Exception {
274+
//test select ALL L0 files when L0 files count > MIN_FILES_L0_KEY
275+
Configuration conf = HBaseConfiguration.create();
276+
conf.setInt(StripeStoreConfig.MIN_FILES_L0_KEY, 4);
277+
StripeCompactionPolicy policy = createPolicy(conf);
278+
StripeCompactionPolicy.StripeInformationProvider si = createStripesWithSizes(10, 10L,
279+
new Long[] { 5L, 1L, 1L }, new Long[] { 3L, 2L, 2L }, new Long[] { 3L, 2L, 2L });
280+
StripeCompactionPolicy.StripeCompactionRequest cr = policy.selectCompaction(si, al(), false);
281+
assertNotNull(cr);
282+
assertEquals(10, cr.getRequest().getFiles().size());
283+
verifyCollectionsEqual(si.getLevel0Files(), cr.getRequest().getFiles());
284+
285+
// test select partial L0 files when size of L0 files > HBASE_HSTORE_COMPACTION_MAX_SIZE_KEY
286+
conf.setLong(HBASE_HSTORE_COMPACTION_MAX_SIZE_KEY, 100L);
287+
policy = createPolicy(conf);
288+
si = createStripesWithSizes(5, 50L,
289+
new Long[] { 5L, 1L, 1L }, new Long[] { 3L, 2L, 2L }, new Long[] { 3L, 2L, 2L });
290+
cr = policy.selectCompaction(si, al(), false);
291+
assertNotNull(cr);
292+
assertEquals(2, cr.getRequest().getFiles().size());
293+
verifyCollectionsEqual(si.getLevel0Files().subList(0, 2), cr.getRequest().getFiles());
294+
295+
// test select partial L0 files when count of L0 files > MAX_FILES_KEY
296+
conf.setInt(MAX_FILES_KEY, 6);
297+
conf.setLong(HBASE_HSTORE_COMPACTION_MAX_SIZE_KEY, 1000L);
298+
policy = createPolicy(conf);
299+
si = createStripesWithSizes(10, 10L,
300+
new Long[] { 5L, 1L, 1L }, new Long[] { 3L, 2L, 2L }, new Long[] { 3L, 2L, 2L });
301+
cr = policy.selectCompaction(si, al(), false);
302+
assertNotNull(cr);
303+
assertEquals(6, cr.getRequest().getFiles().size());
304+
verifyCollectionsEqual(si.getLevel0Files().subList(0, 6), cr.getRequest().getFiles());
305+
}
306+
271307
@Test
272308
public void testExistingStripesFromL0() throws Exception {
273309
Configuration conf = HBaseConfiguration.create();

0 commit comments

Comments
 (0)