1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.regionserver.compactions;
20
21 import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_KEY;
22
23 import java.io.IOException;
24 import java.util.ArrayList;
25 import java.util.Collection;
26 import java.util.List;
27
28 import com.google.common.collect.ImmutableList;
29
30 import org.apache.commons.logging.Log;
31 import org.apache.commons.logging.LogFactory;
32 import org.apache.hadoop.conf.Configuration;
33 import org.apache.hadoop.fs.Path;
34 import org.apache.hadoop.hbase.KeyValue.KVComparator;
35 import org.apache.hadoop.hbase.classification.InterfaceAudience;
36 import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
37 import org.apache.hadoop.hbase.regionserver.StoreFile;
38 import org.apache.hadoop.hbase.regionserver.StoreUtils;
39 import org.apache.hadoop.hbase.regionserver.StripeStoreConfig;
40 import org.apache.hadoop.hbase.regionserver.StripeStoreFlusher;
41 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
42 import org.apache.hadoop.hbase.security.User;
43 import org.apache.hadoop.hbase.util.Bytes;
44 import org.apache.hadoop.hbase.util.ConcatenatedLists;
45 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
46 import org.apache.hadoop.hbase.util.Pair;
47
48
49
50
51 @InterfaceAudience.Private
52 public class StripeCompactionPolicy extends CompactionPolicy {
53 private final static Log LOG = LogFactory.getLog(StripeCompactionPolicy.class);
54
55 private ExploringCompactionPolicy stripePolicy = null;
56
57 private StripeStoreConfig config;
58
59 public StripeCompactionPolicy(
60 Configuration conf, StoreConfigInformation storeConfigInfo, StripeStoreConfig config) {
61 super(conf, storeConfigInfo);
62 this.config = config;
63 stripePolicy = new ExploringCompactionPolicy(conf, storeConfigInfo);
64 }
65
66 public List<StoreFile> preSelectFilesForCoprocessor(StripeInformationProvider si,
67 List<StoreFile> filesCompacting) {
68
69
70
71 ArrayList<StoreFile> candidateFiles = new ArrayList<StoreFile>(si.getStorefiles());
72 candidateFiles.removeAll(filesCompacting);
73 return candidateFiles;
74 }
75
76 public StripeCompactionRequest createEmptyRequest(
77 StripeInformationProvider si, CompactionRequest request) {
78
79 if (si.getStripeCount() > 0) {
80 return new BoundaryStripeCompactionRequest(request, si.getStripeBoundaries());
81 }
82 Pair<Long, Integer> targetKvsAndCount = estimateTargetKvs(
83 request.getFiles(), this.config.getInitialCount());
84 return new SplitStripeCompactionRequest(
85 request, OPEN_KEY, OPEN_KEY, targetKvsAndCount.getSecond(), targetKvsAndCount.getFirst());
86 }
87
88 public StripeStoreFlusher.StripeFlushRequest selectFlush(KVComparator comparator,
89 StripeInformationProvider si, int kvCount) {
90 if (this.config.isUsingL0Flush()) {
91
92 return new StripeStoreFlusher.StripeFlushRequest(comparator);
93 }
94 if (si.getStripeCount() == 0) {
95
96 int initialCount = this.config.getInitialCount();
97 return new StripeStoreFlusher.SizeStripeFlushRequest(comparator, initialCount,
98 kvCount / initialCount);
99 }
100
101 return new StripeStoreFlusher.BoundaryStripeFlushRequest(comparator, si.getStripeBoundaries());
102 }
103
104 public StripeCompactionRequest selectCompaction(StripeInformationProvider si,
105 List<StoreFile> filesCompacting, boolean isOffpeak) throws IOException {
106
107
108 if (!filesCompacting.isEmpty()) {
109 LOG.debug("Not selecting compaction: " + filesCompacting.size() + " files compacting");
110 return null;
111 }
112
113
114
115
116
117
118
119 Collection<StoreFile> allFiles = si.getStorefiles();
120 if (StoreUtils.hasReferences(allFiles)) {
121 LOG.debug("There are references in the store; compacting all files");
122 long targetKvs = estimateTargetKvs(allFiles, config.getInitialCount()).getFirst();
123 SplitStripeCompactionRequest request = new SplitStripeCompactionRequest(
124 allFiles, OPEN_KEY, OPEN_KEY, targetKvs);
125 request.setMajorRangeFull();
126 request.getRequest().setAfterSplit(true);
127 return request;
128 }
129
130 int stripeCount = si.getStripeCount();
131 List<StoreFile> l0Files = si.getLevel0Files();
132
133
134 boolean shouldCompactL0 = (this.config.getLevel0MinFiles() <= l0Files.size());
135 if (stripeCount == 0) {
136 if (!shouldCompactL0) return null;
137 return selectNewStripesCompaction(si);
138 }
139
140 boolean canDropDeletesNoL0 = l0Files.size() == 0;
141 if (shouldCompactL0) {
142 if (!canDropDeletesNoL0) {
143
144 StripeCompactionRequest result = selectSingleStripeCompaction(
145 si, true, canDropDeletesNoL0, isOffpeak);
146 if (result != null) return result;
147 }
148 LOG.debug("Selecting L0 compaction with " + l0Files.size() + " files");
149 return new BoundaryStripeCompactionRequest(l0Files, si.getStripeBoundaries());
150 }
151
152
153 StripeCompactionRequest result = selectExpiredMergeCompaction(si, canDropDeletesNoL0);
154 if (result != null) return result;
155
156
157
158 return selectSingleStripeCompaction(si, false, canDropDeletesNoL0, isOffpeak);
159 }
160
161 public boolean needsCompactions(StripeInformationProvider si, List<StoreFile> filesCompacting) {
162
163 return filesCompacting.isEmpty()
164 && (StoreUtils.hasReferences(si.getStorefiles())
165 || (si.getLevel0Files().size() >= this.config.getLevel0MinFiles())
166 || needsSingleStripeCompaction(si));
167 }
168
169 @Override
170 public boolean shouldPerformMajorCompaction(Collection<StoreFile> filesToCompact)
171 throws IOException {
172 return false;
173 }
174
175 @Override
176 public boolean throttleCompaction(long compactionSize) {
177 return compactionSize > comConf.getThrottlePoint();
178 }
179
180
181
182
183
184 protected boolean needsSingleStripeCompaction(StripeInformationProvider si) {
185 int minFiles = this.config.getStripeCompactMinFiles();
186 for (List<StoreFile> stripe : si.getStripes()) {
187 if (stripe.size() >= minFiles) return true;
188 }
189 return false;
190 }
191
192 protected StripeCompactionRequest selectSingleStripeCompaction(StripeInformationProvider si,
193 boolean includeL0, boolean canDropDeletesWithoutL0, boolean isOffpeak) throws IOException {
194 ArrayList<ImmutableList<StoreFile>> stripes = si.getStripes();
195
196 int bqIndex = -1;
197 List<StoreFile> bqSelection = null;
198 int stripeCount = stripes.size();
199 long bqTotalSize = -1;
200 for (int i = 0; i < stripeCount; ++i) {
201
202
203 List<StoreFile> selection = selectSimpleCompaction(stripes.get(i),
204 !canDropDeletesWithoutL0 && includeL0, isOffpeak);
205 if (selection.isEmpty()) continue;
206 long size = 0;
207 for (StoreFile sf : selection) {
208 size += sf.getReader().length();
209 }
210 if (bqSelection == null || selection.size() > bqSelection.size() ||
211 (selection.size() == bqSelection.size() && size < bqTotalSize)) {
212 bqSelection = selection;
213 bqIndex = i;
214 bqTotalSize = size;
215 }
216 }
217 if (bqSelection == null) {
218 LOG.debug("No good compaction is possible in any stripe");
219 return null;
220 }
221 List<StoreFile> filesToCompact = new ArrayList<StoreFile>(bqSelection);
222
223 int targetCount = 1;
224 long targetKvs = Long.MAX_VALUE;
225 boolean hasAllFiles = filesToCompact.size() == stripes.get(bqIndex).size();
226 String splitString = "";
227 if (hasAllFiles && bqTotalSize >= config.getSplitSize()) {
228 if (includeL0) {
229
230
231 return null;
232 }
233 Pair<Long, Integer> kvsAndCount = estimateTargetKvs(filesToCompact, config.getSplitCount());
234 targetKvs = kvsAndCount.getFirst();
235 targetCount = kvsAndCount.getSecond();
236 splitString = "; the stripe will be split into at most "
237 + targetCount + " stripes with " + targetKvs + " target KVs";
238 }
239
240 LOG.debug("Found compaction in a stripe with end key ["
241 + Bytes.toString(si.getEndRow(bqIndex)) + "], with "
242 + filesToCompact.size() + " files of total size " + bqTotalSize + splitString);
243
244
245 StripeCompactionRequest req;
246 if (includeL0) {
247 assert hasAllFiles;
248 List<StoreFile> l0Files = si.getLevel0Files();
249 LOG.debug("Adding " + l0Files.size() + " files to compaction to be able to drop deletes");
250 ConcatenatedLists<StoreFile> sfs = new ConcatenatedLists<StoreFile>();
251 sfs.addSublist(filesToCompact);
252 sfs.addSublist(l0Files);
253 req = new BoundaryStripeCompactionRequest(sfs, si.getStripeBoundaries());
254 } else {
255 req = new SplitStripeCompactionRequest(
256 filesToCompact, si.getStartRow(bqIndex), si.getEndRow(bqIndex), targetCount, targetKvs);
257 }
258 if (hasAllFiles && (canDropDeletesWithoutL0 || includeL0)) {
259 req.setMajorRange(si.getStartRow(bqIndex), si.getEndRow(bqIndex));
260 }
261 req.getRequest().setOffPeak(isOffpeak);
262 return req;
263 }
264
265
266
267
268
269
270
271 private List<StoreFile> selectSimpleCompaction(
272 List<StoreFile> sfs, boolean allFilesOnly, boolean isOffpeak) {
273 int minFilesLocal = Math.max(
274 allFilesOnly ? sfs.size() : 0, this.config.getStripeCompactMinFiles());
275 int maxFilesLocal = Math.max(this.config.getStripeCompactMaxFiles(), minFilesLocal);
276 return stripePolicy.applyCompactionPolicy(sfs, false, isOffpeak, minFilesLocal, maxFilesLocal);
277 }
278
279
280
281
282
283
284
285
286 private StripeCompactionRequest selectCompactionOfAllFiles(StripeInformationProvider si,
287 int targetStripeCount, long targetSize) {
288 Collection<StoreFile> allFiles = si.getStorefiles();
289 SplitStripeCompactionRequest request = new SplitStripeCompactionRequest(
290 allFiles, OPEN_KEY, OPEN_KEY, targetStripeCount, targetSize);
291 request.setMajorRangeFull();
292 LOG.debug("Selecting a compaction that includes all " + allFiles.size() + " files");
293 return request;
294 }
295
296 private StripeCompactionRequest selectNewStripesCompaction(StripeInformationProvider si) {
297 List<StoreFile> l0Files = si.getLevel0Files();
298 Pair<Long, Integer> kvsAndCount = estimateTargetKvs(l0Files, config.getInitialCount());
299 LOG.debug("Creating " + kvsAndCount.getSecond() + " initial stripes with "
300 + kvsAndCount.getFirst() + " kvs each via L0 compaction of " + l0Files.size() + " files");
301 SplitStripeCompactionRequest request = new SplitStripeCompactionRequest(
302 si.getLevel0Files(), OPEN_KEY, OPEN_KEY, kvsAndCount.getSecond(), kvsAndCount.getFirst());
303 request.setMajorRangeFull();
304 return request;
305 }
306
307 private StripeCompactionRequest selectExpiredMergeCompaction(
308 StripeInformationProvider si, boolean canDropDeletesNoL0) {
309 long cfTtl = this.storeConfigInfo.getStoreFileTtl();
310 if (cfTtl == Long.MAX_VALUE) {
311 return null;
312 }
313 long timestampCutoff = EnvironmentEdgeManager.currentTime() - cfTtl;
314
315 int start = -1, bestStart = -1, length = 0, bestLength = 0;
316 ArrayList<ImmutableList<StoreFile>> stripes = si.getStripes();
317 OUTER: for (int i = 0; i < stripes.size(); ++i) {
318 for (StoreFile storeFile : stripes.get(i)) {
319 if (storeFile.getReader().getMaxTimestamp() < timestampCutoff) continue;
320
321 if (length > bestLength) {
322 bestStart = start;
323 bestLength = length;
324 }
325 start = -1;
326 length = 0;
327 continue OUTER;
328 }
329 if (start == -1) {
330 start = i;
331 }
332 ++length;
333 }
334 if (length > bestLength) {
335 bestStart = start;
336 bestLength = length;
337 }
338 if (bestLength == 0) return null;
339 if (bestLength == 1) {
340
341
342
343
344 if (bestStart == (stripes.size() - 1)) return null;
345 ++bestLength;
346 }
347 LOG.debug("Merging " + bestLength + " stripes to delete expired store files");
348 int endIndex = bestStart + bestLength - 1;
349 ConcatenatedLists<StoreFile> sfs = new ConcatenatedLists<StoreFile>();
350 sfs.addAllSublists(stripes.subList(bestStart, endIndex + 1));
351 SplitStripeCompactionRequest result = new SplitStripeCompactionRequest(sfs,
352 si.getStartRow(bestStart), si.getEndRow(endIndex), 1, Long.MAX_VALUE);
353 if (canDropDeletesNoL0) {
354 result.setMajorRangeFull();
355 }
356 return result;
357 }
358
359 private static long getTotalKvCount(final Collection<StoreFile> candidates) {
360 long totalSize = 0;
361 for (StoreFile storeFile : candidates) {
362 totalSize += storeFile.getReader().getEntries();
363 }
364 return totalSize;
365 }
366
367 public static long getTotalFileSize(final Collection<StoreFile> candidates) {
368 long totalSize = 0;
369 for (StoreFile storeFile : candidates) {
370 totalSize += storeFile.getReader().length();
371 }
372 return totalSize;
373 }
374
375 private Pair<Long, Integer> estimateTargetKvs(Collection<StoreFile> files, double splitCount) {
376
377
378
379
380 long totalSize = getTotalFileSize(files);
381 long targetPartSize = config.getSplitPartSize();
382 assert targetPartSize > 0 && splitCount > 0;
383 double ratio = totalSize / (splitCount * targetPartSize);
384 while (ratio > 1.0) {
385
386 double newRatio = totalSize / ((splitCount + 1.0) * targetPartSize);
387 if ((1.0 / newRatio) >= ratio) break;
388 ratio = newRatio;
389 splitCount += 1.0;
390 }
391 long kvCount = (long)(getTotalKvCount(files) / splitCount);
392 return new Pair<Long, Integer>(kvCount, (int)Math.ceil(splitCount));
393 }
394
395
396 public abstract static class StripeCompactionRequest {
397 protected CompactionRequest request;
398 protected byte[] majorRangeFromRow = null, majorRangeToRow = null;
399
400 public List<Path> execute(StripeCompactor compactor,
401 ThroughputController throughputController) throws IOException {
402 return execute(compactor, throughputController, null);
403 }
404
405
406
407
408
409
410 public abstract List<Path> execute(StripeCompactor compactor,
411 ThroughputController throughputController, User user) throws IOException;
412
413 public StripeCompactionRequest(CompactionRequest request) {
414 this.request = request;
415 }
416
417
418
419
420
421
422
423 public void setMajorRange(byte[] startRow, byte[] endRow) {
424 this.majorRangeFromRow = startRow;
425 this.majorRangeToRow = endRow;
426 }
427
428 public CompactionRequest getRequest() {
429 return this.request;
430 }
431
432 public void setRequest(CompactionRequest request) {
433 assert request != null;
434 this.request = request;
435 this.majorRangeFromRow = this.majorRangeToRow = null;
436 }
437 }
438
439
440
441
442
443 private static class BoundaryStripeCompactionRequest extends StripeCompactionRequest {
444 private final List<byte[]> targetBoundaries;
445
446
447
448
449
450 public BoundaryStripeCompactionRequest(CompactionRequest request,
451 List<byte[]> targetBoundaries) {
452 super(request);
453 this.targetBoundaries = targetBoundaries;
454 }
455
456 public BoundaryStripeCompactionRequest(Collection<StoreFile> files,
457 List<byte[]> targetBoundaries) {
458 this(new CompactionRequest(files), targetBoundaries);
459 }
460
461 @Override
462 public List<Path> execute(StripeCompactor compactor,
463 ThroughputController throughputController, User user) throws IOException {
464 return compactor.compact(this.request, this.targetBoundaries, this.majorRangeFromRow,
465 this.majorRangeToRow, throughputController, user);
466 }
467 }
468
469
470
471
472
473
474
475 private static class SplitStripeCompactionRequest extends StripeCompactionRequest {
476 private final byte[] startRow, endRow;
477 private final int targetCount;
478 private final long targetKvs;
479
480
481
482
483
484
485
486
487
488 public SplitStripeCompactionRequest(CompactionRequest request,
489 byte[] startRow, byte[] endRow, int targetCount, long targetKvs) {
490 super(request);
491 this.startRow = startRow;
492 this.endRow = endRow;
493 this.targetCount = targetCount;
494 this.targetKvs = targetKvs;
495 }
496
497 public SplitStripeCompactionRequest(
498 CompactionRequest request, byte[] startRow, byte[] endRow, long targetKvs) {
499 this(request, startRow, endRow, Integer.MAX_VALUE, targetKvs);
500 }
501
502 public SplitStripeCompactionRequest(
503 Collection<StoreFile> files, byte[] startRow, byte[] endRow, long targetKvs) {
504 this(files, startRow, endRow, Integer.MAX_VALUE, targetKvs);
505 }
506
507 public SplitStripeCompactionRequest(Collection<StoreFile> files,
508 byte[] startRow, byte[] endRow, int targetCount, long targetKvs) {
509 this(new CompactionRequest(files), startRow, endRow, targetCount, targetKvs);
510 }
511
512 @Override
513 public List<Path> execute(StripeCompactor compactor,
514 ThroughputController throughputController, User user) throws IOException {
515 return compactor.compact(this.request, this.targetCount, this.targetKvs, this.startRow,
516 this.endRow, this.majorRangeFromRow, this.majorRangeToRow, throughputController, user);
517 }
518
519
520
521 public void setMajorRangeFull() {
522 setMajorRange(this.startRow, this.endRow);
523 }
524 }
525
526
527 public static interface StripeInformationProvider {
528 public Collection<StoreFile> getStorefiles();
529
530
531
532
533
534
535 public byte[] getStartRow(int stripeIndex);
536
537
538
539
540
541
542 public byte[] getEndRow(int stripeIndex);
543
544
545
546
547 public List<StoreFile> getLevel0Files();
548
549
550
551
552 public List<byte[]> getStripeBoundaries();
553
554
555
556
557 public ArrayList<ImmutableList<StoreFile>> getStripes();
558
559
560
561
562 public int getStripeCount();
563 }
564 }