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;
20
21 import java.io.IOException;
22 import java.util.ArrayList;
23 import java.util.Arrays;
24 import java.util.Collection;
25 import java.util.Collections;
26 import java.util.Comparator;
27 import java.util.HashMap;
28 import java.util.Iterator;
29 import java.util.List;
30 import java.util.Map;
31 import java.util.TreeMap;
32
33 import org.apache.commons.logging.Log;
34 import org.apache.commons.logging.LogFactory;
35 import org.apache.hadoop.conf.Configuration;
36 import org.apache.hadoop.hbase.Cell;
37 import org.apache.hadoop.hbase.HConstants;
38 import org.apache.hadoop.hbase.KeyValue;
39 import org.apache.hadoop.hbase.KeyValue.KVComparator;
40 import org.apache.hadoop.hbase.classification.InterfaceAudience;
41 import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy;
42 import org.apache.hadoop.hbase.util.Bytes;
43 import org.apache.hadoop.hbase.util.ConcatenatedLists;
44 import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
45
46 import com.google.common.collect.ImmutableCollection;
47 import com.google.common.collect.ImmutableList;
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65 @InterfaceAudience.Private
66 public class StripeStoreFileManager
67 implements StoreFileManager, StripeCompactionPolicy.StripeInformationProvider {
68 private static final Log LOG = LogFactory.getLog(StripeStoreFileManager.class);
69
70
71
72
73 public static final byte[] STRIPE_START_KEY = Bytes.toBytes("STRIPE_START_KEY");
74 public static final byte[] STRIPE_END_KEY = Bytes.toBytes("STRIPE_END_KEY");
75
76 private final static Bytes.RowEndKeyComparator MAP_COMPARATOR = new Bytes.RowEndKeyComparator();
77
78
79
80
81 public final static byte[] OPEN_KEY = HConstants.EMPTY_BYTE_ARRAY;
82 final static byte[] INVALID_KEY = null;
83
84
85
86
87
88 private static class State {
89
90
91
92
93
94 public byte[][] stripeEndRows = new byte[0][];
95
96
97
98
99
100
101 public ArrayList<ImmutableList<StoreFile>> stripeFiles
102 = new ArrayList<ImmutableList<StoreFile>>();
103
104 public ImmutableList<StoreFile> level0Files = ImmutableList.<StoreFile>of();
105
106
107 public ImmutableList<StoreFile> allFilesCached = ImmutableList.<StoreFile>of();
108 private ImmutableList<StoreFile> allCompactedFilesCached = ImmutableList.<StoreFile>of();
109 }
110 private State state = null;
111
112
113 private HashMap<StoreFile, byte[]> fileStarts = new HashMap<StoreFile, byte[]>();
114 private HashMap<StoreFile, byte[]> fileEnds = new HashMap<StoreFile, byte[]>();
115
116
117
118 private static final byte[] INVALID_KEY_IN_MAP = new byte[0];
119
120 private final KVComparator kvComparator;
121 private StripeStoreConfig config;
122
123 private final int blockingFileCount;
124
125 public StripeStoreFileManager(
126 KVComparator kvComparator, Configuration conf, StripeStoreConfig config) {
127 this.kvComparator = kvComparator;
128 this.config = config;
129 this.blockingFileCount = conf.getInt(
130 HStore.BLOCKING_STOREFILES_KEY, HStore.DEFAULT_BLOCKING_STOREFILE_COUNT);
131 }
132
133 @Override
134 public void loadFiles(List<StoreFile> storeFiles) {
135 loadUnclassifiedStoreFiles(storeFiles);
136 }
137
138 @Override
139 public Collection<StoreFile> getStorefiles() {
140 return state.allFilesCached;
141 }
142
143 @Override
144 public Collection<StoreFile> getCompactedfiles() {
145 return state.allCompactedFilesCached;
146 }
147
148 @Override
149 public void insertNewFiles(Collection<StoreFile> sfs) throws IOException {
150 CompactionOrFlushMergeCopy cmc = new CompactionOrFlushMergeCopy(true);
151
152 cmc.mergeResults(null, sfs);
153 debugDumpState("Added new files");
154 }
155
156 @Override
157 public ImmutableCollection<StoreFile> clearFiles() {
158 ImmutableCollection<StoreFile> result = state.allFilesCached;
159 this.state = new State();
160 this.fileStarts.clear();
161 this.fileEnds.clear();
162 return result;
163 }
164
165 @Override
166 public ImmutableCollection<StoreFile> clearCompactedFiles() {
167 ImmutableCollection<StoreFile> result = state.allCompactedFilesCached;
168 this.state = new State();
169 return result;
170 }
171
172 @Override
173 public int getStorefileCount() {
174 return state.allFilesCached.size();
175 }
176
177
178
179 @Override
180 public Iterator<StoreFile> getCandidateFilesForRowKeyBefore(final KeyValue targetKey) {
181 KeyBeforeConcatenatedLists result = new KeyBeforeConcatenatedLists();
182
183 result.addSublist(state.level0Files);
184 if (!state.stripeFiles.isEmpty()) {
185 int lastStripeIndex = findStripeForRow(targetKey.getRow(), false);
186 for (int stripeIndex = lastStripeIndex; stripeIndex >= 0; --stripeIndex) {
187 result.addSublist(state.stripeFiles.get(stripeIndex));
188 }
189 }
190 return result.iterator();
191 }
192
193
194
195
196 @Override
197 public Iterator<StoreFile> updateCandidateFilesForRowKeyBefore(
198 Iterator<StoreFile> candidateFiles, final KeyValue targetKey, final Cell candidate) {
199 KeyBeforeConcatenatedLists.Iterator original =
200 (KeyBeforeConcatenatedLists.Iterator)candidateFiles;
201 assert original != null;
202 ArrayList<List<StoreFile>> components = original.getComponents();
203 for (int firstIrrelevant = 0; firstIrrelevant < components.size(); ++firstIrrelevant) {
204 StoreFile sf = components.get(firstIrrelevant).get(0);
205 byte[] endKey = endOf(sf);
206
207
208
209 if (!isInvalid(endKey) && !isOpen(endKey)
210 && (nonOpenRowCompare(endKey, targetKey.getRow()) <= 0)) {
211 original.removeComponents(firstIrrelevant);
212 break;
213 }
214 }
215 return original;
216 }
217
218 @Override
219
220
221
222
223
224
225
226 public byte[] getSplitPoint() throws IOException {
227 if (this.getStorefileCount() == 0) return null;
228 if (state.stripeFiles.size() <= 1) {
229 return getSplitPointFromAllFiles();
230 }
231 int leftIndex = -1, rightIndex = state.stripeFiles.size();
232 long leftSize = 0, rightSize = 0;
233 long lastLeftSize = 0, lastRightSize = 0;
234 while (rightIndex - 1 != leftIndex) {
235 if (leftSize >= rightSize) {
236 --rightIndex;
237 lastRightSize = getStripeFilesSize(rightIndex);
238 rightSize += lastRightSize;
239 } else {
240 ++leftIndex;
241 lastLeftSize = getStripeFilesSize(leftIndex);
242 leftSize += lastLeftSize;
243 }
244 }
245 if (leftSize == 0 || rightSize == 0) {
246 String errMsg = String.format("Cannot split on a boundary - left index %d size %d, "
247 + "right index %d size %d", leftIndex, leftSize, rightIndex, rightSize);
248 debugDumpState(errMsg);
249 LOG.warn(errMsg);
250 return getSplitPointFromAllFiles();
251 }
252 double ratio = (double)rightSize / leftSize;
253 if (ratio < 1) {
254 ratio = 1 / ratio;
255 }
256 if (config.getMaxSplitImbalance() > ratio) return state.stripeEndRows[leftIndex];
257
258
259
260
261
262 boolean isRightLarger = rightSize >= leftSize;
263 double newRatio = isRightLarger
264 ? getMidStripeSplitRatio(leftSize, rightSize, lastRightSize)
265 : getMidStripeSplitRatio(rightSize, leftSize, lastLeftSize);
266 if (newRatio < 1) {
267 newRatio = 1 / newRatio;
268 }
269 if (newRatio >= ratio) return state.stripeEndRows[leftIndex];
270 LOG.debug("Splitting the stripe - ratio w/o split " + ratio + ", ratio with split "
271 + newRatio + " configured ratio " + config.getMaxSplitImbalance());
272
273 return StoreUtils.getLargestFile(state.stripeFiles.get(
274 isRightLarger ? rightIndex : leftIndex)).getFileSplitPoint(this.kvComparator);
275 }
276
277 private byte[] getSplitPointFromAllFiles() throws IOException {
278 ConcatenatedLists<StoreFile> sfs = new ConcatenatedLists<StoreFile>();
279 sfs.addSublist(state.level0Files);
280 sfs.addAllSublists(state.stripeFiles);
281 if (sfs.isEmpty()) return null;
282 return StoreUtils.getLargestFile(sfs).getFileSplitPoint(this.kvComparator);
283 }
284
285 private double getMidStripeSplitRatio(long smallerSize, long largerSize, long lastLargerSize) {
286 return (double)(largerSize - lastLargerSize / 2f) / (smallerSize + lastLargerSize / 2f);
287 }
288
289 @Override
290 public Collection<StoreFile> getFilesForScanOrGet(byte[] startRow, boolean includeStartRow,
291 byte[] stopRow, boolean includeStopRow) {
292 if (state.stripeFiles.isEmpty()) {
293 return state.level0Files;
294 }
295
296 int firstStripe = findStripeForRow(startRow, true);
297 int lastStripe = findStripeForRow(stopRow, false);
298 assert firstStripe <= lastStripe;
299 if (firstStripe == lastStripe && state.level0Files.isEmpty()) {
300 return state.stripeFiles.get(firstStripe);
301 }
302 if (firstStripe == 0 && lastStripe == (state.stripeFiles.size() - 1)) {
303 return state.allFilesCached;
304 }
305
306 ConcatenatedLists<StoreFile> result = new ConcatenatedLists<StoreFile>();
307 result.addAllSublists(state.stripeFiles.subList(firstStripe, lastStripe + 1));
308 result.addSublist(state.level0Files);
309 return result;
310 }
311
312 @Override
313 public void addCompactionResults(
314 Collection<StoreFile> compactedFiles, Collection<StoreFile> results) throws IOException {
315
316 LOG.debug("Attempting to merge compaction results: " + compactedFiles.size()
317 + " files replaced by " + results.size());
318
319
320 CompactionOrFlushMergeCopy cmc = new CompactionOrFlushMergeCopy(false);
321 cmc.mergeResults(compactedFiles, results);
322 markCompactedAway(compactedFiles);
323 debugDumpState("Merged compaction results");
324 }
325
326
327
328
329
330 private void markCompactedAway(Collection<StoreFile> compactedFiles) {
331 for (StoreFile file : compactedFiles) {
332 file.markCompactedAway();
333 }
334 }
335
336 @Override
337 public void removeCompactedFiles(Collection<StoreFile> compactedFiles) throws IOException {
338
339 LOG.debug("Attempting to delete compaction results: " + compactedFiles.size());
340
341
342 CompactionOrFlushMergeCopy cmc = new CompactionOrFlushMergeCopy(false);
343 cmc.deleteResults(compactedFiles);
344 debugDumpState("Deleted compaction results");
345 }
346
347 @Override
348 public int getStoreCompactionPriority() {
349
350
351
352 int fc = getStorefileCount();
353 if (state.stripeFiles.isEmpty() || (this.blockingFileCount <= fc)) {
354 return this.blockingFileCount - fc;
355 }
356
357
358
359 int l0 = state.level0Files.size(), sc = state.stripeFiles.size();
360 int priority = (int)Math.ceil(((double)(this.blockingFileCount - fc + l0) / sc) - l0);
361 return (priority <= HStore.PRIORITY_USER) ? (HStore.PRIORITY_USER + 1) : priority;
362 }
363
364
365
366
367
368
369 private long getStripeFilesSize(int stripeIndex) {
370 long result = 0;
371 for (StoreFile sf : state.stripeFiles.get(stripeIndex)) {
372 result += sf.getReader().length();
373 }
374 return result;
375 }
376
377
378
379
380
381
382
383
384 private void loadUnclassifiedStoreFiles(List<StoreFile> storeFiles) {
385 LOG.debug("Attempting to load " + storeFiles.size() + " store files.");
386 TreeMap<byte[], ArrayList<StoreFile>> candidateStripes =
387 new TreeMap<byte[], ArrayList<StoreFile>>(MAP_COMPARATOR);
388 ArrayList<StoreFile> level0Files = new ArrayList<StoreFile>();
389
390
391 for (StoreFile sf : storeFiles) {
392 byte[] startRow = startOf(sf), endRow = endOf(sf);
393
394 if (isInvalid(startRow) || isInvalid(endRow)) {
395 insertFileIntoStripe(level0Files, sf);
396 ensureLevel0Metadata(sf);
397 } else if (!isOpen(startRow) && !isOpen(endRow) &&
398 nonOpenRowCompare(startRow, endRow) >= 0) {
399 LOG.error("Unexpected metadata - start row [" + Bytes.toString(startRow) + "], end row ["
400 + Bytes.toString(endRow) + "] in file [" + sf.getPath() + "], pushing to L0");
401 insertFileIntoStripe(level0Files, sf);
402 ensureLevel0Metadata(sf);
403 } else {
404 ArrayList<StoreFile> stripe = candidateStripes.get(endRow);
405 if (stripe == null) {
406 stripe = new ArrayList<StoreFile>();
407 candidateStripes.put(endRow, stripe);
408 }
409 insertFileIntoStripe(stripe, sf);
410 }
411 }
412
413
414
415 boolean hasOverlaps = false;
416 byte[] expectedStartRow = null;
417 Iterator<Map.Entry<byte[], ArrayList<StoreFile>>> entryIter =
418 candidateStripes.entrySet().iterator();
419 while (entryIter.hasNext()) {
420 Map.Entry<byte[], ArrayList<StoreFile>> entry = entryIter.next();
421 ArrayList<StoreFile> files = entry.getValue();
422
423 for (int i = 0; i < files.size(); ++i) {
424 StoreFile sf = files.get(i);
425 byte[] startRow = startOf(sf);
426 if (expectedStartRow == null) {
427 expectedStartRow = startRow;
428 } else if (!rowEquals(expectedStartRow, startRow)) {
429 hasOverlaps = true;
430 LOG.warn("Store file doesn't fit into the tentative stripes - expected to start at ["
431 + Bytes.toString(expectedStartRow) + "], but starts at [" + Bytes.toString(startRow)
432 + "], to L0 it goes");
433 StoreFile badSf = files.remove(i);
434 insertFileIntoStripe(level0Files, badSf);
435 ensureLevel0Metadata(badSf);
436 --i;
437 }
438 }
439
440 byte[] endRow = entry.getKey();
441 if (!files.isEmpty()) {
442 expectedStartRow = endRow;
443 } else {
444 entryIter.remove();
445 }
446 }
447
448
449
450
451
452 if (!candidateStripes.isEmpty()) {
453 StoreFile firstFile = candidateStripes.firstEntry().getValue().get(0);
454 boolean isOpen = isOpen(startOf(firstFile)) && isOpen(candidateStripes.lastKey());
455 if (!isOpen) {
456 LOG.warn("The range of the loaded files does not cover full key space: from ["
457 + Bytes.toString(startOf(firstFile)) + "], to ["
458 + Bytes.toString(candidateStripes.lastKey()) + "]");
459 if (!hasOverlaps) {
460 ensureEdgeStripeMetadata(candidateStripes.firstEntry().getValue(), true);
461 ensureEdgeStripeMetadata(candidateStripes.lastEntry().getValue(), false);
462 } else {
463 LOG.warn("Inconsistent files, everything goes to L0.");
464 for (ArrayList<StoreFile> files : candidateStripes.values()) {
465 for (StoreFile sf : files) {
466 insertFileIntoStripe(level0Files, sf);
467 ensureLevel0Metadata(sf);
468 }
469 }
470 candidateStripes.clear();
471 }
472 }
473 }
474
475
476 State state = new State();
477 state.level0Files = ImmutableList.copyOf(level0Files);
478 state.stripeFiles = new ArrayList<ImmutableList<StoreFile>>(candidateStripes.size());
479 state.stripeEndRows = new byte[Math.max(0, candidateStripes.size() - 1)][];
480 ArrayList<StoreFile> newAllFiles = new ArrayList<StoreFile>(level0Files);
481 int i = candidateStripes.size() - 1;
482 for (Map.Entry<byte[], ArrayList<StoreFile>> entry : candidateStripes.entrySet()) {
483 state.stripeFiles.add(ImmutableList.copyOf(entry.getValue()));
484 newAllFiles.addAll(entry.getValue());
485 if (i > 0) {
486 state.stripeEndRows[state.stripeFiles.size() - 1] = entry.getKey();
487 }
488 --i;
489 }
490 state.allFilesCached = ImmutableList.copyOf(newAllFiles);
491 this.state = state;
492 debugDumpState("Files loaded");
493 }
494
495 private void ensureEdgeStripeMetadata(ArrayList<StoreFile> stripe, boolean isFirst) {
496 HashMap<StoreFile, byte[]> targetMap = isFirst ? fileStarts : fileEnds;
497 for (StoreFile sf : stripe) {
498 targetMap.put(sf, OPEN_KEY);
499 }
500 }
501
502 private void ensureLevel0Metadata(StoreFile sf) {
503 if (!isInvalid(startOf(sf))) this.fileStarts.put(sf, INVALID_KEY_IN_MAP);
504 if (!isInvalid(endOf(sf))) this.fileEnds.put(sf, INVALID_KEY_IN_MAP);
505 }
506
507 private void debugDumpState(String string) {
508 if (!LOG.isDebugEnabled()) return;
509 StringBuilder sb = new StringBuilder();
510 sb.append("\n" + string + "; current stripe state is as such:");
511 sb.append("\n level 0 with ")
512 .append(state.level0Files.size())
513 .append(
514 " files: "
515 + TraditionalBinaryPrefix.long2String(
516 StripeCompactionPolicy.getTotalFileSize(state.level0Files), "", 1) + ";");
517 for (int i = 0; i < state.stripeFiles.size(); ++i) {
518 String endRow = (i == state.stripeEndRows.length)
519 ? "(end)" : "[" + Bytes.toString(state.stripeEndRows[i]) + "]";
520 sb.append("\n stripe ending in ")
521 .append(endRow)
522 .append(" with ")
523 .append(state.stripeFiles.get(i).size())
524 .append(
525 " files: "
526 + TraditionalBinaryPrefix.long2String(
527 StripeCompactionPolicy.getTotalFileSize(state.stripeFiles.get(i)), "", 1) + ";");
528 }
529 sb.append("\n").append(state.stripeFiles.size()).append(" stripes total.");
530 sb.append("\n").append(getStorefileCount()).append(" files total.");
531 LOG.debug(sb.toString());
532 }
533
534
535
536
537 private static final boolean isOpen(byte[] key) {
538 return key != null && key.length == 0;
539 }
540
541
542
543
544 private static final boolean isInvalid(byte[] key) {
545 return key == INVALID_KEY;
546 }
547
548
549
550
551 private final boolean rowEquals(byte[] k1, byte[] k2) {
552 return kvComparator.matchingRows(k1, 0, k1.length, k2, 0, k2.length);
553 }
554
555
556
557
558 private final int nonOpenRowCompare(byte[] k1, byte[] k2) {
559 assert !isOpen(k1) && !isOpen(k2);
560 return kvComparator.compareRows(k1, 0, k1.length, k2, 0, k2.length);
561 }
562
563
564
565
566 private final int findStripeIndexByEndRow(byte[] endRow) {
567 assert !isInvalid(endRow);
568 if (isOpen(endRow)) return state.stripeEndRows.length;
569 return Arrays.binarySearch(state.stripeEndRows, endRow, Bytes.BYTES_COMPARATOR);
570 }
571
572
573
574
575 private final int findStripeForRow(byte[] row, boolean isStart) {
576 if (isStart && row == HConstants.EMPTY_START_ROW) return 0;
577 if (!isStart && row == HConstants.EMPTY_END_ROW) return state.stripeFiles.size() - 1;
578
579
580
581
582
583 return Math.abs(Arrays.binarySearch(state.stripeEndRows, row, Bytes.BYTES_COMPARATOR) + 1);
584 }
585
586 @Override
587 public final byte[] getStartRow(int stripeIndex) {
588 return (stripeIndex == 0 ? OPEN_KEY : state.stripeEndRows[stripeIndex - 1]);
589 }
590
591 @Override
592 public final byte[] getEndRow(int stripeIndex) {
593 return (stripeIndex == state.stripeEndRows.length
594 ? OPEN_KEY : state.stripeEndRows[stripeIndex]);
595 }
596
597
598 private byte[] startOf(StoreFile sf) {
599 byte[] result = this.fileStarts.get(sf);
600 return result == null ? sf.getMetadataValue(STRIPE_START_KEY)
601 : (result == INVALID_KEY_IN_MAP ? INVALID_KEY : result);
602 }
603
604 private byte[] endOf(StoreFile sf) {
605 byte[] result = this.fileEnds.get(sf);
606 return result == null ? sf.getMetadataValue(STRIPE_END_KEY)
607 : (result == INVALID_KEY_IN_MAP ? INVALID_KEY : result);
608 }
609
610
611
612
613
614
615 private static void insertFileIntoStripe(ArrayList<StoreFile> stripe, StoreFile sf) {
616
617
618 for (int insertBefore = 0; ; ++insertBefore) {
619 if (insertBefore == stripe.size()
620 || (StoreFile.Comparators.SEQ_ID.compare(sf, stripe.get(insertBefore)) >= 0)) {
621 stripe.add(insertBefore, sf);
622 break;
623 }
624 }
625 }
626
627
628
629
630
631
632
633
634
635
636 private static class KeyBeforeConcatenatedLists extends ConcatenatedLists<StoreFile> {
637 @Override
638 public java.util.Iterator<StoreFile> iterator() {
639 return new Iterator();
640 }
641
642 public class Iterator extends ConcatenatedLists<StoreFile>.Iterator {
643 public ArrayList<List<StoreFile>> getComponents() {
644 return components;
645 }
646
647 public void removeComponents(int startIndex) {
648 List<List<StoreFile>> subList = components.subList(startIndex, components.size());
649 for (List<StoreFile> entry : subList) {
650 size -= entry.size();
651 }
652 assert size >= 0;
653 subList.clear();
654 }
655
656 @Override
657 public void remove() {
658 if (!this.nextWasCalled) {
659 throw new IllegalStateException("No element to remove");
660 }
661 this.nextWasCalled = false;
662 List<StoreFile> src = components.get(currentComponent);
663 if (src instanceof ImmutableList<?>) {
664 src = new ArrayList<StoreFile>(src);
665 components.set(currentComponent, src);
666 }
667 src.remove(indexWithinComponent);
668 --size;
669 --indexWithinComponent;
670 if (src.isEmpty()) {
671 components.remove(currentComponent);
672 }
673 }
674 }
675 }
676
677
678
679
680
681
682 private class CompactionOrFlushMergeCopy {
683 private ArrayList<List<StoreFile>> stripeFiles = null;
684 private ArrayList<StoreFile> level0Files = null;
685 private ArrayList<byte[]> stripeEndRows = null;
686
687 private Collection<StoreFile> compactedFiles = null;
688 private Collection<StoreFile> results = null;
689
690 private List<StoreFile> l0Results = new ArrayList<StoreFile>();
691 private final boolean isFlush;
692
693 public CompactionOrFlushMergeCopy(boolean isFlush) {
694
695 this.stripeFiles = new ArrayList<List<StoreFile>>(
696 StripeStoreFileManager.this.state.stripeFiles);
697 this.isFlush = isFlush;
698 }
699
700 private void mergeResults(Collection<StoreFile> compactedFiles, Collection<StoreFile> results)
701 throws IOException {
702 assert this.compactedFiles == null && this.results == null;
703 this.compactedFiles = compactedFiles;
704 this.results = results;
705
706 if (!isFlush) removeCompactedFiles();
707 TreeMap<byte[], StoreFile> newStripes = processResults();
708 if (newStripes != null) {
709 processNewCandidateStripes(newStripes);
710 }
711
712 State state = createNewState(false);
713 StripeStoreFileManager.this.state = state;
714 updateMetadataMaps();
715 }
716
717 private void deleteResults(Collection<StoreFile> compactedFiles) throws IOException {
718 this.compactedFiles = compactedFiles;
719
720 State state = createNewState(true);
721 StripeStoreFileManager.this.state = state;
722 updateMetadataMaps();
723 }
724
725 private State createNewState(boolean delCompactedFiles) {
726 State oldState = StripeStoreFileManager.this.state;
727
728 assert oldState.stripeFiles.size() == this.stripeFiles.size() || this.stripeEndRows != null;
729 State newState = new State();
730 newState.level0Files = (this.level0Files == null) ? oldState.level0Files
731 : ImmutableList.copyOf(this.level0Files);
732 newState.stripeEndRows = (this.stripeEndRows == null) ? oldState.stripeEndRows
733 : this.stripeEndRows.toArray(new byte[this.stripeEndRows.size()][]);
734 newState.stripeFiles = new ArrayList<ImmutableList<StoreFile>>(this.stripeFiles.size());
735 for (List<StoreFile> newStripe : this.stripeFiles) {
736 newState.stripeFiles.add(newStripe instanceof ImmutableList<?>
737 ? (ImmutableList<StoreFile>)newStripe : ImmutableList.copyOf(newStripe));
738 }
739
740 List<StoreFile> newAllFiles = new ArrayList<StoreFile>(oldState.allFilesCached);
741 List<StoreFile> newAllCompactedFiles =
742 new ArrayList<StoreFile>(oldState.allCompactedFilesCached);
743 if (!isFlush) {
744 newAllFiles.removeAll(compactedFiles);
745 if (delCompactedFiles) {
746 newAllCompactedFiles.removeAll(compactedFiles);
747 } else {
748 newAllCompactedFiles.addAll(compactedFiles);
749 }
750 }
751 if (results != null) {
752 newAllFiles.addAll(results);
753 }
754 newState.allFilesCached = ImmutableList.copyOf(newAllFiles);
755 newState.allCompactedFilesCached = ImmutableList.copyOf(newAllCompactedFiles);
756 return newState;
757 }
758
759 private void updateMetadataMaps() {
760 StripeStoreFileManager parent = StripeStoreFileManager.this;
761 if (!isFlush) {
762 for (StoreFile sf : this.compactedFiles) {
763 parent.fileStarts.remove(sf);
764 parent.fileEnds.remove(sf);
765 }
766 }
767 if (this.l0Results != null) {
768 for (StoreFile sf : this.l0Results) {
769 parent.ensureLevel0Metadata(sf);
770 }
771 }
772 }
773
774
775
776
777
778 private final ArrayList<StoreFile> getStripeCopy(int index) {
779 List<StoreFile> stripeCopy = this.stripeFiles.get(index);
780 ArrayList<StoreFile> result = null;
781 if (stripeCopy instanceof ImmutableList<?>) {
782 result = new ArrayList<StoreFile>(stripeCopy);
783 this.stripeFiles.set(index, result);
784 } else {
785 result = (ArrayList<StoreFile>)stripeCopy;
786 }
787 return result;
788 }
789
790
791
792
793 private final ArrayList<StoreFile> getLevel0Copy() {
794 if (this.level0Files == null) {
795 this.level0Files = new ArrayList<StoreFile>(StripeStoreFileManager.this.state.level0Files);
796 }
797 return this.level0Files;
798 }
799
800
801
802
803
804
805 private TreeMap<byte[], StoreFile> processResults() throws IOException {
806 TreeMap<byte[], StoreFile> newStripes = null;
807 for (StoreFile sf : this.results) {
808 byte[] startRow = startOf(sf), endRow = endOf(sf);
809 if (isInvalid(endRow) || isInvalid(startRow)) {
810 if (!isFlush) {
811 LOG.warn("The newly compacted file doesn't have stripes set: " + sf.getPath());
812 }
813 insertFileIntoStripe(getLevel0Copy(), sf);
814 this.l0Results.add(sf);
815 continue;
816 }
817 if (!this.stripeFiles.isEmpty()) {
818 int stripeIndex = findStripeIndexByEndRow(endRow);
819 if ((stripeIndex >= 0) && rowEquals(getStartRow(stripeIndex), startRow)) {
820
821 insertFileIntoStripe(getStripeCopy(stripeIndex), sf);
822 continue;
823 }
824 }
825
826
827 if (newStripes == null) {
828 newStripes = new TreeMap<byte[], StoreFile>(MAP_COMPARATOR);
829 }
830 StoreFile oldSf = newStripes.put(endRow, sf);
831 if (oldSf != null) {
832 throw new IOException("Compactor has produced multiple files for the stripe ending in ["
833 + Bytes.toString(endRow) + "], found " + sf.getPath() + " and " + oldSf.getPath());
834 }
835 }
836 return newStripes;
837 }
838
839
840
841
842 private void removeCompactedFiles() throws IOException {
843 for (StoreFile oldFile : this.compactedFiles) {
844 byte[] oldEndRow = endOf(oldFile);
845 List<StoreFile> source = null;
846 if (isInvalid(oldEndRow)) {
847 source = getLevel0Copy();
848 } else {
849 int stripeIndex = findStripeIndexByEndRow(oldEndRow);
850 if (stripeIndex < 0) {
851 throw new IOException("An allegedly compacted file [" + oldFile + "] does not belong"
852 + " to a known stripe (end row - [" + Bytes.toString(oldEndRow) + "])");
853 }
854 source = getStripeCopy(stripeIndex);
855 }
856 if (!source.remove(oldFile)) {
857 throw new IOException("An allegedly compacted file [" + oldFile + "] was not found");
858 }
859 }
860 }
861
862
863
864
865
866
867 private void processNewCandidateStripes(
868 TreeMap<byte[], StoreFile> newStripes) throws IOException {
869
870 boolean hasStripes = !this.stripeFiles.isEmpty();
871 this.stripeEndRows = new ArrayList<byte[]>(
872 Arrays.asList(StripeStoreFileManager.this.state.stripeEndRows));
873 int removeFrom = 0;
874 byte[] firstStartRow = startOf(newStripes.firstEntry().getValue());
875 byte[] lastEndRow = newStripes.lastKey();
876 if (!hasStripes && (!isOpen(firstStartRow) || !isOpen(lastEndRow))) {
877 throw new IOException("Newly created stripes do not cover the entire key space.");
878 }
879
880 boolean canAddNewStripes = true;
881 Collection<StoreFile> filesForL0 = null;
882 if (hasStripes) {
883
884
885 if (isOpen(firstStartRow)) {
886 removeFrom = 0;
887 } else {
888 removeFrom = findStripeIndexByEndRow(firstStartRow);
889 if (removeFrom < 0) throw new IOException("Compaction is trying to add a bad range.");
890 ++removeFrom;
891 }
892 int removeTo = findStripeIndexByEndRow(lastEndRow);
893 if (removeTo < 0) throw new IOException("Compaction is trying to add a bad range.");
894
895 ArrayList<StoreFile> conflictingFiles = new ArrayList<StoreFile>();
896 for (int removeIndex = removeTo; removeIndex >= removeFrom; --removeIndex) {
897 conflictingFiles.addAll(this.stripeFiles.get(removeIndex));
898 }
899 if (!conflictingFiles.isEmpty()) {
900
901
902
903 if (isFlush) {
904 long newSize = StripeCompactionPolicy.getTotalFileSize(newStripes.values());
905 LOG.warn("Stripes were created by a flush, but results of size " + newSize
906 + " cannot be added because the stripes have changed");
907 canAddNewStripes = false;
908 filesForL0 = newStripes.values();
909 } else {
910 long oldSize = StripeCompactionPolicy.getTotalFileSize(conflictingFiles);
911 LOG.info(conflictingFiles.size() + " conflicting files (likely created by a flush) "
912 + " of size " + oldSize + " are moved to L0 due to concurrent stripe change");
913 filesForL0 = conflictingFiles;
914 }
915 if (filesForL0 != null) {
916 for (StoreFile sf : filesForL0) {
917 insertFileIntoStripe(getLevel0Copy(), sf);
918 }
919 l0Results.addAll(filesForL0);
920 }
921 }
922
923 if (canAddNewStripes) {
924
925 int originalCount = this.stripeFiles.size();
926 for (int removeIndex = removeTo; removeIndex >= removeFrom; --removeIndex) {
927 if (removeIndex != originalCount - 1) {
928 this.stripeEndRows.remove(removeIndex);
929 }
930 this.stripeFiles.remove(removeIndex);
931 }
932 }
933 }
934
935 if (!canAddNewStripes) return;
936
937
938 byte[] previousEndRow = null;
939 int insertAt = removeFrom;
940 for (Map.Entry<byte[], StoreFile> newStripe : newStripes.entrySet()) {
941 if (previousEndRow != null) {
942
943 assert !isOpen(previousEndRow);
944 byte[] startRow = startOf(newStripe.getValue());
945 if (!rowEquals(previousEndRow, startRow)) {
946 throw new IOException("The new stripes produced by "
947 + (isFlush ? "flush" : "compaction") + " are not contiguous");
948 }
949 }
950
951 ArrayList<StoreFile> tmp = new ArrayList<StoreFile>();
952 tmp.add(newStripe.getValue());
953 stripeFiles.add(insertAt, tmp);
954 previousEndRow = newStripe.getKey();
955 if (!isOpen(previousEndRow)) {
956 stripeEndRows.add(insertAt, previousEndRow);
957 }
958 ++insertAt;
959 }
960 }
961 }
962
963 @Override
964 public List<StoreFile> getLevel0Files() {
965 return this.state.level0Files;
966 }
967
968 @Override
969 public List<byte[]> getStripeBoundaries() {
970 if (this.state.stripeFiles.isEmpty()) return new ArrayList<byte[]>();
971 ArrayList<byte[]> result = new ArrayList<byte[]>(this.state.stripeEndRows.length + 2);
972 result.add(OPEN_KEY);
973 Collections.addAll(result, this.state.stripeEndRows);
974 result.add(OPEN_KEY);
975 return result;
976 }
977
978 @Override
979 public ArrayList<ImmutableList<StoreFile>> getStripes() {
980 return this.state.stripeFiles;
981 }
982
983 @Override
984 public int getStripeCount() {
985 return this.state.stripeFiles.size();
986 }
987
988 @Override
989 public Collection<StoreFile> getUnneededFiles(long maxTs, List<StoreFile> filesCompacting) {
990
991
992 State state = this.state;
993 Collection<StoreFile> expiredStoreFiles = null;
994 for (ImmutableList<StoreFile> stripe : state.stripeFiles) {
995 expiredStoreFiles = findExpiredFiles(stripe, maxTs, filesCompacting, expiredStoreFiles);
996 }
997 return findExpiredFiles(state.level0Files, maxTs, filesCompacting, expiredStoreFiles);
998 }
999
1000 private Collection<StoreFile> findExpiredFiles(ImmutableList<StoreFile> stripe, long maxTs,
1001 List<StoreFile> filesCompacting, Collection<StoreFile> expiredStoreFiles) {
1002
1003 for (int i = 1; i < stripe.size(); ++i) {
1004 StoreFile sf = stripe.get(i);
1005 synchronized (sf) {
1006 long fileTs = sf.getReader().getMaxTimestamp();
1007 if (fileTs < maxTs && !filesCompacting.contains(sf)) {
1008 LOG.info("Found an expired store file: " + sf.getPath() + " whose maxTimeStamp is "
1009 + fileTs + ", which is below " + maxTs);
1010 if (expiredStoreFiles == null) {
1011 expiredStoreFiles = new ArrayList<StoreFile>();
1012 }
1013 expiredStoreFiles.add(sf);
1014 }
1015 }
1016 }
1017 return expiredStoreFiles;
1018 }
1019
1020 @Override
1021 public double getCompactionPressure() {
1022 State stateLocal = this.state;
1023 if (stateLocal.allFilesCached.size() > blockingFileCount) {
1024
1025 return 2.0;
1026 }
1027 if (stateLocal.stripeFiles.isEmpty()) {
1028 return 0.0;
1029 }
1030 int blockingFilePerStripe = blockingFileCount / stateLocal.stripeFiles.size();
1031
1032
1033 int delta = stateLocal.level0Files.isEmpty() ? 0 : 1;
1034 double max = 0.0;
1035 for (ImmutableList<StoreFile> stripeFile : stateLocal.stripeFiles) {
1036 int stripeFileCount = stripeFile.size();
1037 double normCount =
1038 (double) (stripeFileCount + delta - config.getStripeCompactMinFiles())
1039 / (blockingFilePerStripe - config.getStripeCompactMinFiles());
1040 if (normCount >= 1.0) {
1041
1042
1043 return 1.0;
1044 }
1045 if (normCount > max) {
1046 max = normCount;
1047 }
1048 }
1049 return max;
1050 }
1051
1052 @Override
1053 public Comparator<StoreFile> getStoreFileComparator() {
1054 return StoreFile.Comparators.SEQ_ID;
1055 }
1056 }