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