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