001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.regionserver.compactions; 019 020import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_KEY; 021 022import java.io.IOException; 023import java.util.ArrayList; 024import java.util.Collection; 025import java.util.List; 026import org.apache.hadoop.conf.Configuration; 027import org.apache.hadoop.fs.Path; 028import org.apache.hadoop.hbase.CellComparator; 029import org.apache.hadoop.hbase.regionserver.HStoreFile; 030import org.apache.hadoop.hbase.regionserver.StoreConfigInformation; 031import org.apache.hadoop.hbase.regionserver.StoreUtils; 032import org.apache.hadoop.hbase.regionserver.StripeStoreConfig; 033import org.apache.hadoop.hbase.regionserver.StripeStoreFlusher; 034import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; 035import org.apache.hadoop.hbase.security.User; 036import org.apache.hadoop.hbase.util.Bytes; 037import org.apache.hadoop.hbase.util.ConcatenatedLists; 038import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 039import org.apache.hadoop.hbase.util.Pair; 040import org.apache.yetus.audience.InterfaceAudience; 041import org.slf4j.Logger; 042import org.slf4j.LoggerFactory; 043 044import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; 045 046/** 047 * Stripe store implementation of compaction policy. 048 */ 049@InterfaceAudience.Private 050public class StripeCompactionPolicy extends CompactionPolicy { 051 private final static Logger LOG = LoggerFactory.getLogger(StripeCompactionPolicy.class); 052 // Policy used to compact individual stripes. 053 private ExploringCompactionPolicy stripePolicy = null; 054 055 private StripeStoreConfig config; 056 057 public StripeCompactionPolicy(Configuration conf, StoreConfigInformation storeConfigInfo, 058 StripeStoreConfig config) { 059 super(conf, storeConfigInfo); 060 this.config = config; 061 stripePolicy = new ExploringCompactionPolicy(conf, storeConfigInfo); 062 } 063 064 public List<HStoreFile> preSelectFilesForCoprocessor(StripeInformationProvider si, 065 List<HStoreFile> filesCompacting) { 066 // We sincerely hope nobody is messing with us with their coprocessors. 067 // If they do, they are very likely to shoot themselves in the foot. 068 // We'll just exclude all the filesCompacting from the list. 069 ArrayList<HStoreFile> candidateFiles = new ArrayList<>(si.getStoreFiles()); 070 candidateFiles.removeAll(filesCompacting); 071 return candidateFiles; 072 } 073 074 public StripeCompactionRequest createEmptyRequest(StripeInformationProvider si, 075 CompactionRequestImpl request) { 076 // Treat as L0-ish compaction with fixed set of files, and hope for the best. 077 if (si.getStripeCount() > 0) { 078 return new BoundaryStripeCompactionRequest(request, si.getStripeBoundaries()); 079 } 080 Pair<Long, Integer> targetKvsAndCount = 081 estimateTargetKvs(request.getFiles(), this.config.getInitialCount()); 082 return new SplitStripeCompactionRequest(request, OPEN_KEY, OPEN_KEY, 083 targetKvsAndCount.getSecond(), targetKvsAndCount.getFirst()); 084 } 085 086 public StripeStoreFlusher.StripeFlushRequest selectFlush(CellComparator comparator, 087 StripeInformationProvider si, int kvCount) { 088 if (this.config.isUsingL0Flush()) { 089 // L0 is used, return dumb request. 090 return new StripeStoreFlusher.StripeFlushRequest(comparator); 091 } 092 if (si.getStripeCount() == 0) { 093 // No stripes - start with the requisite count, derive KVs per stripe. 094 int initialCount = this.config.getInitialCount(); 095 return new StripeStoreFlusher.SizeStripeFlushRequest(comparator, initialCount, 096 kvCount / initialCount); 097 } 098 // There are stripes - do according to the boundaries. 099 return new StripeStoreFlusher.BoundaryStripeFlushRequest(comparator, si.getStripeBoundaries()); 100 } 101 102 public StripeCompactionRequest selectCompaction(StripeInformationProvider si, 103 List<HStoreFile> filesCompacting, boolean isOffpeak) throws IOException { 104 // TODO: first cut - no parallel compactions. To have more fine grained control we 105 // probably need structure more sophisticated than a list. 106 if (!filesCompacting.isEmpty()) { 107 LOG.debug("Not selecting compaction: " + filesCompacting.size() + " files compacting"); 108 return null; 109 } 110 111 // We are going to do variations of compaction in strict order of preference. 112 // A better/more advanced approach is to use a heuristic to see which one is "more 113 // necessary" at current time. 114 115 // This can happen due to region split. We can skip it later; for now preserve 116 // compact-all-things behavior. 117 Collection<HStoreFile> allFiles = si.getStoreFiles(); 118 if (StoreUtils.hasReferences(allFiles)) { 119 LOG.debug("There are references in the store; compacting all files"); 120 long targetKvs = estimateTargetKvs(allFiles, config.getInitialCount()).getFirst(); 121 SplitStripeCompactionRequest request = 122 new SplitStripeCompactionRequest(allFiles, OPEN_KEY, OPEN_KEY, targetKvs); 123 request.setMajorRangeFull(); 124 request.getRequest().setAfterSplit(true); 125 return request; 126 } 127 128 int stripeCount = si.getStripeCount(); 129 List<HStoreFile> l0Files = si.getLevel0Files(); 130 131 // See if we need to make new stripes. 132 boolean shouldCompactL0 = 133 this.config.getLevel0MinFiles() <= l0Files.size() || allL0FilesExpired(si); 134 if (stripeCount == 0) { 135 if (!shouldCompactL0) { 136 return null; // nothing to do. 137 } 138 return selectL0OnlyCompaction(si); 139 } 140 141 boolean canDropDeletesNoL0 = l0Files.isEmpty(); 142 if (shouldCompactL0) { 143 if (!canDropDeletesNoL0) { 144 // If we need to compact L0, see if we can add something to it, and drop deletes. 145 StripeCompactionRequest result = 146 selectSingleStripeCompaction(si, !shouldSelectL0Files(si), canDropDeletesNoL0, isOffpeak); 147 if (result != null) { 148 return result; 149 } 150 } 151 LOG.debug("Selecting L0 compaction with " + l0Files.size() + " files"); 152 return selectL0OnlyCompaction(si); 153 } 154 155 // Try to delete fully expired stripes 156 StripeCompactionRequest result = selectExpiredMergeCompaction(si, canDropDeletesNoL0); 157 if (result != null) { 158 return result; 159 } 160 161 // Ok, nothing special here, let's see if we need to do a common compaction. 162 // This will also split the stripes that are too big if needed. 163 return selectSingleStripeCompaction(si, false, canDropDeletesNoL0, isOffpeak); 164 } 165 166 public boolean needsCompactions(StripeInformationProvider si, List<HStoreFile> filesCompacting) { 167 // Approximation on whether we need compaction. 168 return filesCompacting.isEmpty() && (StoreUtils.hasReferences(si.getStoreFiles()) 169 || (si.getLevel0Files().size() >= this.config.getLevel0MinFiles()) 170 || needsSingleStripeCompaction(si) || hasExpiredStripes(si) || allL0FilesExpired(si)); 171 } 172 173 @Override 174 public boolean shouldPerformMajorCompaction(Collection<HStoreFile> filesToCompact) 175 throws IOException { 176 return false; // there's never a major compaction! 177 } 178 179 @Override 180 public boolean throttleCompaction(long compactionSize) { 181 return compactionSize > comConf.getThrottlePoint(); 182 } 183 184 /** 185 * @param si StoreFileManager. 186 * @return Whether any stripe potentially needs compaction. 187 */ 188 protected boolean needsSingleStripeCompaction(StripeInformationProvider si) { 189 int minFiles = this.config.getStripeCompactMinFiles(); 190 for (List<HStoreFile> stripe : si.getStripes()) { 191 if (stripe.size() >= minFiles) return true; 192 } 193 return false; 194 } 195 196 protected StripeCompactionRequest selectSingleStripeCompaction(StripeInformationProvider si, 197 boolean includeL0, boolean canDropDeletesWithoutL0, boolean isOffpeak) throws IOException { 198 ArrayList<ImmutableList<HStoreFile>> stripes = si.getStripes(); 199 200 int bqIndex = -1; 201 List<HStoreFile> bqSelection = null; 202 int stripeCount = stripes.size(); 203 long bqTotalSize = -1; 204 for (int i = 0; i < stripeCount; ++i) { 205 // If we want to compact L0 to drop deletes, we only want whole-stripe compactions. 206 // So, pass includeL0 as 2nd parameter to indicate that. 207 List<HStoreFile> selection = selectSimpleCompaction(stripes.get(i), 208 !canDropDeletesWithoutL0 && includeL0, isOffpeak, false); 209 if (selection.isEmpty()) continue; 210 long size = 0; 211 for (HStoreFile sf : selection) { 212 size += sf.getReader().length(); 213 } 214 if ( 215 bqSelection == null || selection.size() > bqSelection.size() 216 || (selection.size() == bqSelection.size() && size < bqTotalSize) 217 ) { 218 bqSelection = selection; 219 bqIndex = i; 220 bqTotalSize = size; 221 } 222 } 223 if (bqSelection == null) { 224 LOG.debug("No good compaction is possible in any stripe"); 225 return null; 226 } 227 List<HStoreFile> filesToCompact = new ArrayList<>(bqSelection); 228 // See if we can, and need to, split this stripe. 229 int targetCount = 1; 230 long targetKvs = Long.MAX_VALUE; 231 boolean hasAllFiles = filesToCompact.size() == stripes.get(bqIndex).size(); 232 String splitString = ""; 233 if (hasAllFiles && bqTotalSize >= config.getSplitSize()) { 234 if (includeL0) { 235 // We want to avoid the scenario where we compact a stripe w/L0 and then split it. 236 // So, if we might split, don't compact the stripe with L0. 237 return null; 238 } 239 Pair<Long, Integer> kvsAndCount = estimateTargetKvs(filesToCompact, config.getSplitCount()); 240 targetKvs = kvsAndCount.getFirst(); 241 targetCount = kvsAndCount.getSecond(); 242 splitString = "; the stripe will be split into at most " + targetCount + " stripes with " 243 + targetKvs + " target KVs"; 244 } 245 246 LOG.debug("Found compaction in a stripe with end key [" + Bytes.toString(si.getEndRow(bqIndex)) 247 + "], with " + filesToCompact.size() + " files of total size " + bqTotalSize + splitString); 248 249 // See if we can drop deletes. 250 StripeCompactionRequest req; 251 if (includeL0) { 252 assert hasAllFiles; 253 List<HStoreFile> l0Files = si.getLevel0Files(); 254 LOG.debug("Adding " + l0Files.size() + " files to compaction to be able to drop deletes"); 255 ConcatenatedLists<HStoreFile> sfs = new ConcatenatedLists<>(); 256 sfs.addSublist(filesToCompact); 257 sfs.addSublist(l0Files); 258 req = new BoundaryStripeCompactionRequest(sfs, si.getStripeBoundaries()); 259 } else { 260 req = new SplitStripeCompactionRequest(filesToCompact, si.getStartRow(bqIndex), 261 si.getEndRow(bqIndex), targetCount, targetKvs); 262 } 263 if (hasAllFiles && (canDropDeletesWithoutL0 || includeL0)) { 264 req.setMajorRange(si.getStartRow(bqIndex), si.getEndRow(bqIndex)); 265 } 266 req.getRequest().setOffPeak(isOffpeak); 267 return req; 268 } 269 270 /** 271 * Selects the compaction of a single stripe using default policy. 272 * @param sfs Files. 273 * @param allFilesOnly Whether a compaction of all-or-none files is needed. 274 * @return The resulting selection. 275 */ 276 private List<HStoreFile> selectSimpleCompaction(List<HStoreFile> sfs, boolean allFilesOnly, 277 boolean isOffpeak, boolean forceCompact) { 278 int minFilesLocal = 279 Math.max(allFilesOnly ? sfs.size() : 0, this.config.getStripeCompactMinFiles()); 280 int maxFilesLocal = Math.max(this.config.getStripeCompactMaxFiles(), minFilesLocal); 281 List<HStoreFile> selected = 282 stripePolicy.applyCompactionPolicy(sfs, false, isOffpeak, minFilesLocal, maxFilesLocal); 283 if (forceCompact && (selected == null || selected.isEmpty()) && !sfs.isEmpty()) { 284 return stripePolicy.selectCompactFiles(sfs, maxFilesLocal, isOffpeak); 285 } 286 return selected; 287 } 288 289 private boolean shouldSelectL0Files(StripeInformationProvider si) { 290 return si.getLevel0Files().size() > this.config.getStripeCompactMaxFiles() 291 || getTotalFileSize(si.getLevel0Files()) > comConf.getMaxCompactSize(); 292 } 293 294 private StripeCompactionRequest selectL0OnlyCompaction(StripeInformationProvider si) { 295 List<HStoreFile> l0Files = si.getLevel0Files(); 296 List<HStoreFile> selectedFiles = l0Files; 297 if (shouldSelectL0Files(si)) { 298 selectedFiles = selectSimpleCompaction(l0Files, false, false, true); 299 assert !selectedFiles.isEmpty() : "Selected L0 files should not be empty"; 300 } 301 StripeCompactionRequest request; 302 if (si.getStripeCount() == 0) { 303 Pair<Long, Integer> estimate = estimateTargetKvs(selectedFiles, config.getInitialCount()); 304 long targetKvs = estimate.getFirst(); 305 int targetCount = estimate.getSecond(); 306 request = 307 new SplitStripeCompactionRequest(selectedFiles, OPEN_KEY, OPEN_KEY, targetCount, targetKvs); 308 if (selectedFiles.size() == l0Files.size()) { 309 ((SplitStripeCompactionRequest) request).setMajorRangeFull(); // L0 only, can drop deletes. 310 } 311 LOG.debug("Creating {} initial stripes with {} kvs each via L0 compaction of {}/{} files", 312 targetCount, targetKvs, selectedFiles.size(), l0Files.size()); 313 } else { 314 request = new BoundaryStripeCompactionRequest(selectedFiles, si.getStripeBoundaries()); 315 LOG.debug("Boundary L0 compaction of {}/{} files", selectedFiles.size(), l0Files.size()); 316 } 317 return request; 318 } 319 320 private StripeCompactionRequest selectExpiredMergeCompaction(StripeInformationProvider si, 321 boolean canDropDeletesNoL0) { 322 long cfTtl = this.storeConfigInfo.getStoreFileTtl(); 323 if (cfTtl == Long.MAX_VALUE) { 324 return null; // minversion might be set, cannot delete old files 325 } 326 long timestampCutoff = EnvironmentEdgeManager.currentTime() - cfTtl; 327 // Merge the longest sequence of stripes where all files have expired, if any. 328 int start = -1, bestStart = -1, length = 0, bestLength = 0; 329 ArrayList<ImmutableList<HStoreFile>> stripes = si.getStripes(); 330 OUTER: for (int i = 0; i < stripes.size(); ++i) { 331 for (HStoreFile storeFile : stripes.get(i)) { 332 if (storeFile.getReader().getMaxTimestamp() < timestampCutoff) continue; 333 // Found non-expired file, this stripe has to stay. 334 if (length > bestLength) { 335 bestStart = start; 336 bestLength = length; 337 } 338 start = -1; 339 length = 0; 340 continue OUTER; 341 } 342 if (start == -1) { 343 start = i; 344 } 345 ++length; 346 } 347 if (length > bestLength) { 348 bestStart = start; 349 bestLength = length; 350 } 351 if (bestLength == 0) return null; 352 if (bestLength == 1) { 353 // This is currently inefficient. If only one stripe expired, we will rewrite some 354 // entire stripe just to delete some expired files because we rely on metadata and it 355 // cannot simply be updated in an old file. When we either determine stripe dynamically 356 // or move metadata to manifest, we can just drop the "expired stripes". 357 if (bestStart == (stripes.size() - 1)) return null; 358 ++bestLength; 359 } 360 LOG.debug("Merging " + bestLength + " stripes to delete expired store files"); 361 int endIndex = bestStart + bestLength - 1; 362 ConcatenatedLists<HStoreFile> sfs = new ConcatenatedLists<>(); 363 sfs.addAllSublists(stripes.subList(bestStart, endIndex + 1)); 364 SplitStripeCompactionRequest result = new SplitStripeCompactionRequest(sfs, 365 si.getStartRow(bestStart), si.getEndRow(endIndex), 1, Long.MAX_VALUE); 366 if (canDropDeletesNoL0) { 367 result.setMajorRangeFull(); 368 } 369 return result; 370 } 371 372 protected boolean hasExpiredStripes(StripeInformationProvider si) { 373 // Find if exists a stripe where all files have expired, if any. 374 ArrayList<ImmutableList<HStoreFile>> stripes = si.getStripes(); 375 for (ImmutableList<HStoreFile> stripe : stripes) { 376 if (allFilesExpired(stripe)) { 377 return true; 378 } 379 } 380 return false; 381 } 382 383 protected boolean allL0FilesExpired(StripeInformationProvider si) { 384 return allFilesExpired(si.getLevel0Files()); 385 } 386 387 private boolean allFilesExpired(final List<HStoreFile> storeFiles) { 388 if (storeFiles == null || storeFiles.isEmpty()) { 389 return false; 390 } 391 long cfTtl = this.storeConfigInfo.getStoreFileTtl(); 392 if (cfTtl == Long.MAX_VALUE) { 393 return false; // minversion might be set, cannot delete old files 394 } 395 long timestampCutoff = EnvironmentEdgeManager.currentTime() - cfTtl; 396 for (HStoreFile storeFile : storeFiles) { 397 // Check store file is not empty and has not expired 398 if ( 399 storeFile.getReader().getMaxTimestamp() >= timestampCutoff 400 && storeFile.getReader().getEntries() != 0 401 ) { 402 return false; 403 } 404 } 405 return true; 406 } 407 408 private static long getTotalKvCount(final Collection<HStoreFile> candidates) { 409 long totalSize = 0; 410 for (HStoreFile storeFile : candidates) { 411 totalSize += storeFile.getReader().getEntries(); 412 } 413 return totalSize; 414 } 415 416 public static long getTotalFileSize(final Collection<HStoreFile> candidates) { 417 long totalSize = 0; 418 for (HStoreFile storeFile : candidates) { 419 totalSize += storeFile.getReader().length(); 420 } 421 return totalSize; 422 } 423 424 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "FL_FLOATS_AS_LOOP_COUNTERS", 425 justification = "valid usage") 426 private Pair<Long, Integer> estimateTargetKvs(Collection<HStoreFile> files, double splitCount) { 427 // If the size is larger than what we target, we don't want to split into proportionally 428 // larger parts and then have to split again very soon. So, we will increase the multiplier 429 // by one until we get small enough parts. E.g. 5Gb stripe that should have been split into 430 // 2 parts when it was 3Gb will be split into 3x1.67Gb parts, rather than 2x2.5Gb parts. 431 long totalSize = getTotalFileSize(files); 432 long targetPartSize = config.getSplitPartSize(); 433 assert targetPartSize > 0 && splitCount > 0; 434 double ratio = totalSize / (splitCount * targetPartSize); // ratio of real to desired size 435 while (ratio > 1.0) { 436 // Ratio of real to desired size if we increase the multiplier. 437 double newRatio = totalSize / ((splitCount + 1.0) * targetPartSize); 438 if ((1.0 / newRatio) >= ratio) { 439 // New ratio is < 1.0, but further than the last one. 440 break; 441 } 442 ratio = newRatio; 443 splitCount += 1.0; 444 } 445 long kvCount = (long) (getTotalKvCount(files) / splitCount); 446 return new Pair<>(kvCount, (int) Math.ceil(splitCount)); 447 } 448 449 /** Stripe compaction request wrapper. */ 450 public abstract static class StripeCompactionRequest { 451 protected CompactionRequestImpl request; 452 protected byte[] majorRangeFromRow = null, majorRangeToRow = null; 453 454 public List<Path> execute(StripeCompactor compactor, ThroughputController throughputController) 455 throws IOException { 456 return execute(compactor, throughputController, null); 457 } 458 459 /** 460 * Executes the request against compactor (essentially, just calls correct overload of compact 461 * method), to simulate more dynamic dispatch. 462 * @param compactor Compactor. 463 * @return result of compact(...) 464 */ 465 public abstract List<Path> execute(StripeCompactor compactor, 466 ThroughputController throughputController, User user) throws IOException; 467 468 public StripeCompactionRequest(CompactionRequestImpl request) { 469 this.request = request; 470 } 471 472 /** 473 * Sets compaction "major range". Major range is the key range for which all the files are 474 * included, so they can be treated like major-compacted files. 475 * @param startRow Left boundary, inclusive. 476 * @param endRow Right boundary, exclusive. 477 */ 478 public void setMajorRange(byte[] startRow, byte[] endRow) { 479 this.majorRangeFromRow = startRow; 480 this.majorRangeToRow = endRow; 481 } 482 483 public CompactionRequestImpl getRequest() { 484 return this.request; 485 } 486 487 public void setRequest(CompactionRequestImpl request) { 488 assert request != null; 489 this.request = request; 490 this.majorRangeFromRow = this.majorRangeToRow = null; 491 } 492 } 493 494 /** 495 * Request for stripe compactor that will cause it to split the source files into several separate 496 * files at the provided boundaries. 497 */ 498 private static class BoundaryStripeCompactionRequest extends StripeCompactionRequest { 499 private final List<byte[]> targetBoundaries; 500 501 /** 502 * @param request Original request. 503 * @param targetBoundaries New files should be written with these boundaries. 504 */ 505 public BoundaryStripeCompactionRequest(CompactionRequestImpl request, 506 List<byte[]> targetBoundaries) { 507 super(request); 508 this.targetBoundaries = targetBoundaries; 509 } 510 511 public BoundaryStripeCompactionRequest(Collection<HStoreFile> files, 512 List<byte[]> targetBoundaries) { 513 this(new CompactionRequestImpl(files), targetBoundaries); 514 } 515 516 @Override 517 public List<Path> execute(StripeCompactor compactor, ThroughputController throughputController, 518 User user) throws IOException { 519 return compactor.compact(this.request, this.targetBoundaries, this.majorRangeFromRow, 520 this.majorRangeToRow, throughputController, user); 521 } 522 } 523 524 /** 525 * Request for stripe compactor that will cause it to split the source files into several separate 526 * files into based on key-value count, as well as file count limit. Most of the files will be 527 * roughly the same size. The last file may be smaller or larger depending on the interplay of the 528 * amount of data and maximum number of files allowed. 529 */ 530 private static class SplitStripeCompactionRequest extends StripeCompactionRequest { 531 private final byte[] startRow, endRow; 532 private final int targetCount; 533 private final long targetKvs; 534 535 /** 536 * @param request Original request. 537 * @param startRow Left boundary of the range to compact, inclusive. 538 * @param endRow Right boundary of the range to compact, exclusive. 539 * @param targetCount The maximum number of stripe to compact into. 540 * @param targetKvs The KV count of each segment. If targetKvs*targetCount is less than total 541 * number of kvs, all the overflow data goes into the last stripe. 542 */ 543 public SplitStripeCompactionRequest(CompactionRequestImpl request, byte[] startRow, 544 byte[] endRow, int targetCount, long targetKvs) { 545 super(request); 546 this.startRow = startRow; 547 this.endRow = endRow; 548 this.targetCount = targetCount; 549 this.targetKvs = targetKvs; 550 } 551 552 public SplitStripeCompactionRequest(Collection<HStoreFile> files, byte[] startRow, 553 byte[] endRow, long targetKvs) { 554 this(files, startRow, endRow, Integer.MAX_VALUE, targetKvs); 555 } 556 557 public SplitStripeCompactionRequest(Collection<HStoreFile> files, byte[] startRow, 558 byte[] endRow, int targetCount, long targetKvs) { 559 this(new CompactionRequestImpl(files), startRow, endRow, targetCount, targetKvs); 560 } 561 562 @Override 563 public List<Path> execute(StripeCompactor compactor, ThroughputController throughputController, 564 User user) throws IOException { 565 return compactor.compact(this.request, this.targetCount, this.targetKvs, this.startRow, 566 this.endRow, this.majorRangeFromRow, this.majorRangeToRow, throughputController, user); 567 } 568 569 /** 570 * Set major range of the compaction to the entire compaction range. See 571 * {@link #setMajorRange(byte[], byte[])}. 572 */ 573 public void setMajorRangeFull() { 574 setMajorRange(this.startRow, this.endRow); 575 } 576 } 577 578 /** The information about stripes that the policy needs to do its stuff */ 579 public static interface StripeInformationProvider { 580 public Collection<HStoreFile> getStoreFiles(); 581 582 /** 583 * Gets the start row for a given stripe. 584 * @param stripeIndex Stripe index. 585 * @return Start row. May be an open key. 586 */ 587 public byte[] getStartRow(int stripeIndex); 588 589 /** 590 * Gets the end row for a given stripe. 591 * @param stripeIndex Stripe index. 592 * @return End row. May be an open key. 593 */ 594 public byte[] getEndRow(int stripeIndex); 595 596 /** Returns Level 0 files. */ 597 public List<HStoreFile> getLevel0Files(); 598 599 /** Returns All stripe boundaries; including the open ones on both ends. */ 600 public List<byte[]> getStripeBoundaries(); 601 602 /** Returns The stripes. */ 603 public ArrayList<ImmutableList<HStoreFile>> getStripes(); 604 605 /** Returns Stripe count. */ 606 public int getStripeCount(); 607 } 608}