View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
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   * Stripe implementation of StoreFileManager.
51   * Not thread safe - relies on external locking (in HStore). Collections that this class
52   * returns are immutable or unique to the call, so they should be safe.
53   * Stripe store splits the key space of the region into non-overlapping stripes, as well as
54   * some recent files that have all the keys (level 0). Each stripe contains a set of files.
55   * When L0 is compacted, it's split into the files corresponding to existing stripe boundaries,
56   * that can thus be added to stripes.
57   * When scan or get happens, it only has to read the files from the corresponding stripes.
58   * See StripeCompationPolicy on how the stripes are determined; this class doesn't care.
59   *
60   * This class should work together with StripeCompactionPolicy and StripeCompactor.
61   * With regard to how they work, we make at least the following (reasonable) assumptions:
62   *  - Compaction produces one file per new stripe (if any); that is easy to change.
63   *  - Compaction has one contiguous set of stripes both in and out, except if L0 is involved.
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     * The file metadata fields that contain the stripe information.
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     * The key value used for range boundary, indicating that the boundary is open (i.e. +-inf).
80     */
81    public final static byte[] OPEN_KEY = HConstants.EMPTY_BYTE_ARRAY;
82    final static byte[] INVALID_KEY = null;
83  
84    /**
85     * The state class. Used solely to replace results atomically during
86     * compactions and avoid complicated error handling.
87     */
88    private static class State {
89      /**
90       * The end rows of each stripe. The last stripe end is always open-ended, so it's not stored
91       * here. It is invariant that the start row of the stripe is the end row of the previous one
92       * (and is an open boundary for the first one).
93       */
94      public byte[][] stripeEndRows = new byte[0][];
95  
96      /**
97       * Files by stripe. Each element of the list corresponds to stripeEndRow element with the
98       * same index, except the last one. Inside each list, the files are in reverse order by
99       * seqNum. Note that the length of this is one higher than that of stripeEndKeys.
100      */
101     public ArrayList<ImmutableList<StoreFile>> stripeFiles
102       = new ArrayList<ImmutableList<StoreFile>>();
103     /** Level 0. The files are in reverse order by seqNum. */
104     public ImmutableList<StoreFile> level0Files = ImmutableList.<StoreFile>of();
105 
106     /** Cached list of all files in the structure, to return from some calls */
107     public ImmutableList<StoreFile> allFilesCached = ImmutableList.<StoreFile>of();
108     private ImmutableList<StoreFile> allCompactedFilesCached = ImmutableList.<StoreFile>of();
109   }
110   private State state = null;
111 
112   /** Cached file metadata (or overrides as the case may be) */
113   private HashMap<StoreFile, byte[]> fileStarts = new HashMap<StoreFile, byte[]>();
114   private HashMap<StoreFile, byte[]> fileEnds = new HashMap<StoreFile, byte[]>();
115   /** Normally invalid key is null, but in the map null is the result for "no key"; so use
116    * the following constant value in these maps instead. Note that this is a constant and
117    * we use it to compare by reference when we read from the map. */
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     // Passing null does not cause NPE??
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   /** See {@link StoreFileManager#getCandidateFilesForRowKeyBefore(KeyValue)}
178    * for details on this methods. */
179   @Override
180   public Iterator<StoreFile> getCandidateFilesForRowKeyBefore(final KeyValue targetKey) {
181     KeyBeforeConcatenatedLists result = new KeyBeforeConcatenatedLists();
182     // Order matters for this call.
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   /** See {@link StoreFileManager#getCandidateFilesForRowKeyBefore(KeyValue)} and
194    * {@link StoreFileManager#updateCandidateFilesForRowKeyBefore(Iterator, KeyValue, Cell)}
195    * for details on this methods. */
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       // Entries are ordered as such: L0, then stripes in reverse order. We never remove
207       // level 0; we remove the stripe, and all subsequent ones, as soon as we find the
208       // first one that cannot possibly have better candidates.
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    * Override of getSplitPoint that determines the split point as the boundary between two
221    * stripes, unless it causes significant imbalance between split sides' sizes. In that
222    * case, the split boundary will be chosen from the middle of one of the stripes to
223    * minimize imbalance.
224    * @return The split point, or null if no split is possible.
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     // If the difference between the sides is too large, we could get the proportional key on
259     // the a stripe to equalize the difference, but there's no proportional key method at the
260     // moment, and it's not extremely important.
261     // See if we can achieve better ratio if we split the bigger side in half.
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     // Ok, we may get better ratio, get it.
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; // There's just L0.
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); // There's just one stripe we need.
301     }
302     if (firstStripe == 0 && lastStripe == (state.stripeFiles.size() - 1)) {
303       return state.allFilesCached; // We need to read all files.
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     // See class comment for the assumptions we make here.
316     LOG.debug("Attempting to merge compaction results: " + compactedFiles.size()
317         + " files replaced by " + results.size());
318     // In order to be able to fail in the middle of the operation, we'll operate on lazy
319     // copies and apply the result at the end.
320     CompactionOrFlushMergeCopy cmc = new CompactionOrFlushMergeCopy(false);
321     cmc.mergeResults(compactedFiles, results);
322     markCompactedAway(compactedFiles);
323     debugDumpState("Merged compaction results");
324   }
325 
326   // Mark the files as compactedAway once the storefiles and compactedfiles list is finalised
327   // Let a background thread close the actual reader on these compacted files and also
328   // ensure to evict the blocks from block cache so that they are no longer in
329   // cache
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     // See class comment for the assumptions we make here.
339     LOG.debug("Attempting to delete compaction results: " + compactedFiles.size());
340     // In order to be able to fail in the middle of the operation, we'll operate on lazy
341     // copies and apply the result at the end.
342     CompactionOrFlushMergeCopy cmc = new CompactionOrFlushMergeCopy(false);
343     cmc.deleteResults(compactedFiles);
344     debugDumpState("Deleted compaction results");
345   }
346 
347   @Override
348   public int getStoreCompactionPriority() {
349     // If there's only L0, do what the default store does.
350     // If we are in critical priority, do the same - we don't want to trump all stores all
351     // the time due to how many files we have.
352     int fc = getStorefileCount();
353     if (state.stripeFiles.isEmpty() || (this.blockingFileCount <= fc)) {
354       return this.blockingFileCount - fc;
355     }
356     // If we are in good shape, we don't want to be trumped by all other stores due to how
357     // many files we have, so do an approximate mapping to normal priority range; L0 counts
358     // for all stripes.
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    * Gets the total size of all files in the stripe.
366    * @param stripeIndex Stripe index.
367    * @return Size.
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    * Loads initial store files that were picked up from some physical location pertaining to
379    * this store (presumably). Unlike adding files after compaction, assumes empty initial
380    * sets, and is forgiving with regard to stripe constraints - at worst, many/all files will
381    * go to level 0.
382    * @param storeFiles Store files to add.
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     // Separate the files into tentative stripes; then validate. Currently, we rely on metadata.
390     // If needed, we could dynamically determine the stripes in future.
391     for (StoreFile sf : storeFiles) {
392       byte[] startRow = startOf(sf), endRow = endOf(sf);
393       // Validate the range and put the files into place.
394       if (isInvalid(startRow) || isInvalid(endRow)) {
395         insertFileIntoStripe(level0Files, sf); // No metadata - goes to L0.
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); // Bad metadata - goes to L0 also.
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     // Possible improvement - for variable-count stripes, if all the files are in L0, we can
413     // instead create single, open-ended stripe with all files.
414 
415     boolean hasOverlaps = false;
416     byte[] expectedStartRow = null; // first stripe can start wherever
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       // Validate the file start rows, and remove the bad ones to level 0.
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; // ensure that first stripe is still consistent
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       // Check if any files from the candidate stripe are valid. If so, add a stripe.
440       byte[] endRow = entry.getKey();
441       if (!files.isEmpty()) {
442         expectedStartRow = endRow; // Next stripe must start exactly at that key.
443       } else {
444         entryIter.remove();
445       }
446     }
447 
448     // In the end, there must be open ends on two sides. If not, and there were no errors i.e.
449     // files are consistent, they might be coming from a split. We will treat the boundaries
450     // as open keys anyway, and log the message.
451     // If there were errors, we'll play it safe and dump everything into L0.
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     // Copy the results into the fields.
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    * Checks whether the key indicates an open interval boundary (i.e. infinity).
536    */
537   private static final boolean isOpen(byte[] key) {
538     return key != null && key.length == 0;
539   }
540 
541   /**
542    * Checks whether the key is invalid (e.g. from an L0 file, or non-stripe-compacted files).
543    */
544   private static final boolean isInvalid(byte[] key) {
545     return key == INVALID_KEY;
546   }
547 
548   /**
549    * Compare two keys for equality.
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    * Compare two keys. Keys must not be open (isOpen(row) == false).
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    * Finds the stripe index by end row.
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    * Finds the stripe index for the stripe containing a row provided externally for get/scan.
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     // If there's an exact match below, a stripe ends at "row". Stripe right boundary is
579     // exclusive, so that means the row is in the next stripe; thus, we need to add one to index.
580     // If there's no match, the return value of binarySearch is (-(insertion point) - 1), where
581     // insertion point is the index of the next greater element, or list size if none. The
582     // insertion point happens to be exactly what we need, so we need to add one to the result.
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    * Inserts a file in the correct place (by seqnum) in a stripe copy.
612    * @param stripe Stripe copy to insert into.
613    * @param sf File to insert.
614    */
615   private static void insertFileIntoStripe(ArrayList<StoreFile> stripe, StoreFile sf) {
616     // The only operation for which sorting of the files matters is KeyBefore. Therefore,
617     // we will store the file in reverse order by seqNum from the outset.
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    * An extension of ConcatenatedLists that has several peculiar properties.
629    * First, one can cut the tail of the logical list by removing last several sub-lists.
630    * Second, items can be removed thru iterator.
631    * Third, if the sub-lists are immutable, they are replaced with mutable copies when needed.
632    * On average KeyBefore operation will contain half the stripes as potential candidates,
633    * but will quickly cut down on them as it finds something in the more likely ones; thus,
634    * the above allow us to avoid unnecessary copying of a bunch of lists.
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); // indexWithinComponent is already -1 here.
672         }
673       }
674     }
675   }
676 
677   /**
678    * Non-static helper class for merging compaction or flush results.
679    * Since we want to merge them atomically (more or less), it operates on lazy copies,
680    * then creates a new state object and puts it in place.
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       // Create a lazy mutable copy (other fields are so lazy they start out as nulls).
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       // Do logical processing.
706       if (!isFlush) removeCompactedFiles();
707       TreeMap<byte[], StoreFile> newStripes = processResults();
708       if (newStripes != null) {
709         processNewCandidateStripes(newStripes);
710       }
711       // Create new state and update parent.
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       // Create new state and update parent.
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       // Stripe count should be the same unless the end rows changed.
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      * @param index Index of the stripe we need.
776      * @return A lazy stripe copy from current stripes.
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      * @return A lazy L0 copy from current state.
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      * Process new files, and add them either to the structure of existing stripes,
802      * or to the list of new candidate stripes.
803      * @return New candidate stripes.
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             // Simple/common case - add file to an existing stripe.
821             insertFileIntoStripe(getStripeCopy(stripeIndex), sf);
822             continue;
823           }
824         }
825 
826         // Make a new candidate stripe.
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      * Remove compacted files.
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      * See {@link #addCompactionResults(Collection, Collection)} - updates the stripe list with
864      * new candidate stripes/removes old stripes; produces new set of stripe end rows.
865      * @param newStripes  New stripes - files by end row.
866      */
867     private void processNewCandidateStripes(
868         TreeMap<byte[], StoreFile> newStripes) throws IOException {
869       // Validate that the removed and added aggregate ranges still make for a full key space.
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         // Determine which stripes will need to be removed because they conflict with new stripes.
884         // The new boundaries should match old stripe boundaries, so we should get exact matches.
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         // See if there are files in the stripes we are trying to replace.
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           // This can be caused by two things - concurrent flush into stripes, or a bug.
901           // Unfortunately, we cannot tell them apart without looking at timing or something
902           // like that. We will assume we are dealing with a flush and dump it into L0.
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           // Remove old empty stripes.
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; // Files were already put into L0.
936 
937       // Now, insert new stripes. The total ranges match, so we can insert where we removed.
938       byte[] previousEndRow = null;
939       int insertAt = removeFrom;
940       for (Map.Entry<byte[], StoreFile> newStripe : newStripes.entrySet()) {
941         if (previousEndRow != null) {
942           // Validate that the ranges are contiguous.
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         // Add the new stripe.
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     // 1) We can never get rid of the last file which has the maximum seqid in a stripe.
991     // 2) Files that are not the latest can't become one due to (1), so the rest are fair game.
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     // Order by seqnum is reversed.
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       // just a hit to tell others that we have reached the blocking file count.
1025       return 2.0;
1026     }
1027     if (stateLocal.stripeFiles.isEmpty()) {
1028       return 0.0;
1029     }
1030     int blockingFilePerStripe = blockingFileCount / stateLocal.stripeFiles.size();
1031     // do not calculate L0 separately because data will be moved to stripe quickly and in most cases
1032     // we flush data to stripe directly.
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         // This could happen if stripe is not split evenly. Do not return values that larger than
1042         // 1.0 because we have not reached the blocking file count actually.
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 }