|
19 | 19 | package org.apache.hadoop.hbase.regionserver.compactions; |
20 | 20 |
|
21 | 21 | import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_KEY; |
22 | | - |
23 | 22 | import java.io.IOException; |
24 | 23 | import java.util.ArrayList; |
25 | 24 | import java.util.Collection; |
@@ -130,27 +129,33 @@ public StripeCompactionRequest selectCompaction(StripeInformationProvider si, |
130 | 129 | List<HStoreFile> l0Files = si.getLevel0Files(); |
131 | 130 |
|
132 | 131 | // See if we need to make new stripes. |
133 | | - boolean shouldCompactL0 = (this.config.getLevel0MinFiles() <= l0Files.size()); |
| 132 | + boolean shouldCompactL0 = this.config.getLevel0MinFiles() <= l0Files.size(); |
134 | 133 | 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); |
137 | 138 | } |
138 | 139 |
|
139 | 140 | boolean canDropDeletesNoL0 = l0Files.isEmpty(); |
140 | 141 | if (shouldCompactL0) { |
141 | 142 | if (!canDropDeletesNoL0) { |
142 | 143 | // 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 | + } |
146 | 149 | } |
147 | 150 | LOG.debug("Selecting L0 compaction with " + l0Files.size() + " files"); |
148 | | - return new BoundaryStripeCompactionRequest(l0Files, si.getStripeBoundaries()); |
| 151 | + return selectL0OnlyCompaction(si); |
149 | 152 | } |
150 | 153 |
|
151 | 154 | // Try to delete fully expired stripes |
152 | 155 | StripeCompactionRequest result = selectExpiredMergeCompaction(si, canDropDeletesNoL0); |
153 | | - if (result != null) return result; |
| 156 | + if (result != null) { |
| 157 | + return result; |
| 158 | + } |
154 | 159 |
|
155 | 160 | // Ok, nothing special here, let's see if we need to do a common compaction. |
156 | 161 | // This will also split the stripes that are too big if needed. |
@@ -200,7 +205,7 @@ protected StripeCompactionRequest selectSingleStripeCompaction(StripeInformation |
200 | 205 | // If we want to compact L0 to drop deletes, we only want whole-stripe compactions. |
201 | 206 | // So, pass includeL0 as 2nd parameter to indicate that. |
202 | 207 | List<HStoreFile> selection = selectSimpleCompaction(stripes.get(i), |
203 | | - !canDropDeletesWithoutL0 && includeL0, isOffpeak); |
| 208 | + !canDropDeletesWithoutL0 && includeL0, isOffpeak, false); |
204 | 209 | if (selection.isEmpty()) continue; |
205 | 210 | long size = 0; |
206 | 211 | for (HStoreFile sf : selection) { |
@@ -268,21 +273,46 @@ protected StripeCompactionRequest selectSingleStripeCompaction(StripeInformation |
268 | 273 | * @return The resulting selection. |
269 | 274 | */ |
270 | 275 | private List<HStoreFile> selectSimpleCompaction( |
271 | | - List<HStoreFile> sfs, boolean allFilesOnly, boolean isOffpeak) { |
| 276 | + List<HStoreFile> sfs, boolean allFilesOnly, boolean isOffpeak, boolean forceCompact) { |
272 | 277 | int minFilesLocal = Math.max( |
273 | 278 | allFilesOnly ? sfs.size() : 0, this.config.getStripeCompactMinFiles()); |
274 | 279 | 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(); |
276 | 291 | } |
277 | 292 |
|
278 | | - private StripeCompactionRequest selectNewStripesCompaction(StripeInformationProvider si) { |
| 293 | + private StripeCompactionRequest selectL0OnlyCompaction(StripeInformationProvider si) { |
279 | 294 | 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 | + } |
286 | 316 | return request; |
287 | 317 | } |
288 | 318 |
|
|
0 commit comments