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