001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.regionserver.wal;
019
020import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
021
022import java.util.ArrayList;
023import java.util.Collections;
024import java.util.HashMap;
025import java.util.List;
026import java.util.Map;
027import java.util.Set;
028import java.util.TreeMap;
029import java.util.concurrent.ConcurrentHashMap;
030import java.util.concurrent.ConcurrentMap;
031import java.util.stream.Collectors;
032import org.apache.hadoop.hbase.HConstants;
033import org.apache.hadoop.hbase.util.Bytes;
034import org.apache.hadoop.hbase.util.ImmutableByteArray;
035import org.apache.yetus.audience.InterfaceAudience;
036import org.slf4j.Logger;
037import org.slf4j.LoggerFactory;
038
039/**
040 * Accounting of sequence ids per region and then by column family. So we can keep our accounting
041 * current, call startCacheFlush and then finishedCacheFlush or abortCacheFlush so this instance can
042 * keep abreast of the state of sequence id persistence. Also call update per append.
043 * <p>
044 * For the implementation, we assume that all the {@code encodedRegionName} passed in are gotten by
045 * {@link org.apache.hadoop.hbase.client.RegionInfo#getEncodedNameAsBytes()}. So it is safe to use
046 * it as a hash key. And for family name, we use {@link ImmutableByteArray} as key. This is because
047 * hash based map is much faster than RBTree or CSLM and here we are on the critical write path. See
048 * HBASE-16278 for more details.
049 * </p>
050 */
051@InterfaceAudience.Private
052class SequenceIdAccounting {
053  private static final Logger LOG = LoggerFactory.getLogger(SequenceIdAccounting.class);
054
055  /**
056   * This lock ties all operations on {@link SequenceIdAccounting#flushingSequenceIds} and
057   * {@link #lowestUnflushedSequenceIds} Maps. {@link #lowestUnflushedSequenceIds} has the
058   * lowest outstanding sequence ids EXCEPT when flushing. When we flush, the current
059   * lowest set for the region/column family are moved (atomically because of this lock) to
060   * {@link #flushingSequenceIds}.
061   * 
062   * <p>The two Maps are tied by this locking object EXCEPT when we go to update the lowest
063   * entry; see {@link #lowestUnflushedSequenceIds}. In here is a putIfAbsent call on
064   * {@link #lowestUnflushedSequenceIds}. In this latter case, we will add this lowest
065   * sequence id if we find that there is no entry for the current column family. There will be no
066   * entry only if we just came up OR we have moved aside current set of lowest sequence ids
067   * because the current set are being flushed (by putting them into {@link #flushingSequenceIds}).
068   * This is how we pick up the next 'lowest' sequence id per region per column family to be used
069   * figuring what is in the next flush.
070   */
071  private final Object tieLock = new Object();
072
073  /**
074   * Map of encoded region names and family names to their OLDEST -- i.e. their first,
075   * the longest-lived, their 'earliest', the 'lowest' -- sequence id.
076   *
077   * <p>When we flush, the current lowest sequence ids get cleared and added to
078   * {@link #flushingSequenceIds}. The next append that comes in, is then added
079   * here to {@link #lowestUnflushedSequenceIds} as the next lowest sequenceid.
080   *
081   * <p>If flush fails, currently server is aborted so no need to restore previous sequence ids.
082   * <p>Needs to be concurrent Maps because we use putIfAbsent updating oldest.
083   */
084  private final ConcurrentMap<byte[], ConcurrentMap<ImmutableByteArray, Long>>
085    lowestUnflushedSequenceIds = new ConcurrentHashMap<>();
086
087  /**
088   * Map of encoded region names and family names to their lowest or OLDEST sequence/edit id
089   * currently being flushed out to hfiles. Entries are moved here from
090   * {@link #lowestUnflushedSequenceIds} while the lock {@link #tieLock} is held
091   * (so movement between the Maps is atomic).
092   */
093  private final Map<byte[], Map<ImmutableByteArray, Long>> flushingSequenceIds = new HashMap<>();
094
095  /**
096   * <p>
097   * Map of region encoded names to the latest/highest region sequence id. Updated on each call to
098   * append.
099   * </p>
100   * <p>
101   * This map uses byte[] as the key, and uses reference equality. It works in our use case as we
102   * use {@link org.apache.hadoop.hbase.client.RegionInfo#getEncodedNameAsBytes()} as keys. For a
103   * given region, it always returns the same array.
104   * </p>
105   */
106  private Map<byte[], Long> highestSequenceIds = new HashMap<>();
107
108  /**
109   * Returns the lowest unflushed sequence id for the region.
110   * @return Lowest outstanding unflushed sequenceid for <code>encodedRegionName</code>. Will
111   * return {@link HConstants#NO_SEQNUM} when none.
112   */
113  long getLowestSequenceId(final byte[] encodedRegionName) {
114    synchronized (this.tieLock) {
115      Map<?, Long> m = this.flushingSequenceIds.get(encodedRegionName);
116      long flushingLowest = m != null ? getLowestSequenceId(m) : Long.MAX_VALUE;
117      m = this.lowestUnflushedSequenceIds.get(encodedRegionName);
118      long unflushedLowest = m != null ? getLowestSequenceId(m) : HConstants.NO_SEQNUM;
119      return Math.min(flushingLowest, unflushedLowest);
120    }
121  }
122
123  /**
124   * @return Lowest outstanding unflushed sequenceid for <code>encodedRegionname</code> and
125   *         <code>familyName</code>. Returned sequenceid may be for an edit currently being
126   *         flushed.
127   */
128  long getLowestSequenceId(final byte[] encodedRegionName, final byte[] familyName) {
129    ImmutableByteArray familyNameWrapper = ImmutableByteArray.wrap(familyName);
130    synchronized (this.tieLock) {
131      Map<ImmutableByteArray, Long> m = this.flushingSequenceIds.get(encodedRegionName);
132      if (m != null) {
133        Long lowest = m.get(familyNameWrapper);
134        if (lowest != null) {
135          return lowest;
136        }
137      }
138      m = this.lowestUnflushedSequenceIds.get(encodedRegionName);
139      if (m != null) {
140        Long lowest = m.get(familyNameWrapper);
141        if (lowest != null) {
142          return lowest;
143        }
144      }
145    }
146    return HConstants.NO_SEQNUM;
147  }
148
149  /**
150   * Reset the accounting of highest sequenceid by regionname.
151   * @return Return the previous accounting Map of regions to the last sequence id written into
152   * each.
153   */
154  Map<byte[], Long> resetHighest() {
155    Map<byte[], Long> old = this.highestSequenceIds;
156    this.highestSequenceIds = new HashMap<>();
157    return old;
158  }
159
160  /**
161   * We've been passed a new sequenceid for the region. Set it as highest seen for this region and
162   * if we are to record oldest, or lowest sequenceids, save it as oldest seen if nothing
163   * currently older.
164   * @param encodedRegionName
165   * @param families
166   * @param sequenceid
167   * @param lowest Whether to keep running account of oldest sequence id.
168   */
169  void update(byte[] encodedRegionName, Set<byte[]> families, long sequenceid,
170      final boolean lowest) {
171    Long l = Long.valueOf(sequenceid);
172    this.highestSequenceIds.put(encodedRegionName, l);
173    if (lowest) {
174      ConcurrentMap<ImmutableByteArray, Long> m = getOrCreateLowestSequenceIds(encodedRegionName);
175      for (byte[] familyName : families) {
176        m.putIfAbsent(ImmutableByteArray.wrap(familyName), l);
177      }
178    }
179  }
180
181  /**
182   * Clear all the records of the given region as it is going to be closed.
183   * <p/>
184   * We will call this once we get the region close marker. We need this because that, if we use
185   * Durability.ASYNC_WAL, after calling startCacheFlush, we may still get some ongoing wal entries
186   * that has not been processed yet, this will lead to orphan records in the
187   * lowestUnflushedSequenceIds and then cause too many WAL files.
188   * <p/>
189   * See HBASE-23157 for more details.
190   */
191  void onRegionClose(byte[] encodedRegionName) {
192    synchronized (tieLock) {
193      this.lowestUnflushedSequenceIds.remove(encodedRegionName);
194      Map<ImmutableByteArray, Long> flushing = this.flushingSequenceIds.remove(encodedRegionName);
195      if (flushing != null) {
196        LOG.warn("Still have flushing records when closing {}, {}",
197          Bytes.toString(encodedRegionName),
198          flushing.entrySet().stream().map(e -> e.getKey().toString() + "->" + e.getValue())
199            .collect(Collectors.joining(",", "{", "}")));
200      }
201    }
202    this.highestSequenceIds.remove(encodedRegionName);
203  }
204
205  /**
206   * Update the store sequence id, e.g., upon executing in-memory compaction
207   */
208  void updateStore(byte[] encodedRegionName, byte[] familyName, Long sequenceId,
209      boolean onlyIfGreater) {
210    if (sequenceId == null) {
211      return;
212    }
213    Long highest = this.highestSequenceIds.get(encodedRegionName);
214    if (highest == null || sequenceId > highest) {
215      this.highestSequenceIds.put(encodedRegionName, sequenceId);
216    }
217    ImmutableByteArray familyNameWrapper = ImmutableByteArray.wrap(familyName);
218    synchronized (this.tieLock) {
219      ConcurrentMap<ImmutableByteArray, Long> m = getOrCreateLowestSequenceIds(encodedRegionName);
220      boolean replaced = false;
221      while (!replaced) {
222        Long oldSeqId = m.get(familyNameWrapper);
223        if (oldSeqId == null) {
224          m.put(familyNameWrapper, sequenceId);
225          replaced = true;
226        } else if (onlyIfGreater) {
227          if (sequenceId > oldSeqId) {
228            replaced = m.replace(familyNameWrapper, oldSeqId, sequenceId);
229          } else {
230            return;
231          }
232        } else { // replace even if sequence id is not greater than oldSeqId
233          m.put(familyNameWrapper, sequenceId);
234          return;
235        }
236      }
237    }
238  }
239
240  ConcurrentMap<ImmutableByteArray, Long> getOrCreateLowestSequenceIds(byte[] encodedRegionName) {
241    // Intentionally, this access is done outside of this.regionSequenceIdLock. Done per append.
242    return computeIfAbsent(this.lowestUnflushedSequenceIds, encodedRegionName,
243      ConcurrentHashMap::new);
244  }
245
246  /**
247   * @param sequenceids Map to search for lowest value.
248   * @return Lowest value found in <code>sequenceids</code>.
249   */
250  private static long getLowestSequenceId(Map<?, Long> sequenceids) {
251    long lowest = HConstants.NO_SEQNUM;
252    for (Long sid: sequenceids.values()) {
253      if (lowest == HConstants.NO_SEQNUM || sid.longValue() < lowest) {
254        lowest = sid.longValue();
255      }
256    }
257    return lowest;
258  }
259
260  /**
261   * @param src
262   * @return New Map that has same keys as <code>src</code> but instead of a Map for a value, it
263   *         instead has found the smallest sequence id and it returns that as the value instead.
264   */
265  private <T extends Map<?, Long>> Map<byte[], Long> flattenToLowestSequenceId(Map<byte[], T> src) {
266    if (src == null || src.isEmpty()) {
267      return null;
268    }
269    Map<byte[], Long> tgt = new HashMap<>();
270    for (Map.Entry<byte[], T> entry : src.entrySet()) {
271      long lowestSeqId = getLowestSequenceId(entry.getValue());
272      if (lowestSeqId != HConstants.NO_SEQNUM) {
273        tgt.put(entry.getKey(), lowestSeqId);
274      }
275    }
276    return tgt;
277  }
278
279  /**
280   * @param encodedRegionName Region to flush.
281   * @param families Families to flush. May be a subset of all families in the region.
282   * @return Returns {@link HConstants#NO_SEQNUM} if we are flushing the whole region OR if
283   * we are flushing a subset of all families but there are no edits in those families not
284   * being flushed; in other words, this is effectively same as a flush of all of the region
285   * though we were passed a subset of regions. Otherwise, it returns the sequence id of the
286   * oldest/lowest outstanding edit.
287   */
288  Long startCacheFlush(final byte[] encodedRegionName, final Set<byte[]> families) {
289    Map<byte[],Long> familytoSeq = new HashMap<>();
290    for (byte[] familyName : families){
291      familytoSeq.put(familyName,HConstants.NO_SEQNUM);
292    }
293    return startCacheFlush(encodedRegionName,familytoSeq);
294  }
295
296  Long startCacheFlush(final byte[] encodedRegionName, final Map<byte[], Long> familyToSeq) {
297    Map<ImmutableByteArray, Long> oldSequenceIds = null;
298    Long lowestUnflushedInRegion = HConstants.NO_SEQNUM;
299    synchronized (tieLock) {
300      Map<ImmutableByteArray, Long> m = this.lowestUnflushedSequenceIds.get(encodedRegionName);
301      if (m != null) {
302        // NOTE: Removal from this.lowestUnflushedSequenceIds must be done in controlled
303        // circumstance because another concurrent thread now may add sequenceids for this family
304        // (see above in getOrCreateLowestSequenceId). Make sure you are ok with this. Usually it
305        // is fine because updates are blocked when this method is called. Make sure!!!
306        for (Map.Entry<byte[], Long> entry : familyToSeq.entrySet()) {
307          ImmutableByteArray familyNameWrapper = ImmutableByteArray.wrap((byte[]) entry.getKey());
308          Long seqId = null;
309          if(entry.getValue() == HConstants.NO_SEQNUM) {
310            seqId = m.remove(familyNameWrapper);
311          } else {
312            seqId = m.replace(familyNameWrapper, entry.getValue());
313          }
314          if (seqId != null) {
315            if (oldSequenceIds == null) {
316              oldSequenceIds = new HashMap<>();
317            }
318            oldSequenceIds.put(familyNameWrapper, seqId);
319          }
320        }
321        if (oldSequenceIds != null && !oldSequenceIds.isEmpty()) {
322          if (this.flushingSequenceIds.put(encodedRegionName, oldSequenceIds) != null) {
323            LOG.warn("Flushing Map not cleaned up for " + Bytes.toString(encodedRegionName) +
324              ", sequenceid=" + oldSequenceIds);
325          }
326        }
327        if (m.isEmpty()) {
328          // Remove it otherwise it will be in oldestUnflushedStoreSequenceIds for ever
329          // even if the region is already moved to other server.
330          // Do not worry about data racing, we held write lock of region when calling
331          // startCacheFlush, so no one can add value to the map we removed.
332          this.lowestUnflushedSequenceIds.remove(encodedRegionName);
333        } else {
334          // Flushing a subset of the region families. Return the sequence id of the oldest entry.
335          lowestUnflushedInRegion = Collections.min(m.values());
336        }
337      }
338    }
339    // Do this check outside lock.
340    if (oldSequenceIds != null && oldSequenceIds.isEmpty()) {
341      // TODO: if we have no oldStoreSeqNum, and WAL is not disabled, presumably either
342      // the region is already flushing (which would make this call invalid), or there
343      // were no appends after last flush, so why are we starting flush? Maybe we should
344      // assert not empty. Less rigorous, but safer, alternative is telling the caller to stop.
345      // For now preserve old logic.
346      LOG.warn("Couldn't find oldest sequenceid for " + Bytes.toString(encodedRegionName));
347    }
348    return lowestUnflushedInRegion;
349  }
350
351  void completeCacheFlush(byte[] encodedRegionName, long maxFlushedSeqId) {
352    // This is a simple hack to avoid maxFlushedSeqId go backwards.
353    // The system works fine normally, but if we make use of Durability.ASYNC_WAL and we are going
354    // to flush all the stores, the maxFlushedSeqId will be next seq id of the region, but we may
355    // still have some unsynced WAL entries in the ringbuffer after we call startCacheFlush, and
356    // then it will be recorded as the lowestUnflushedSeqId by the above update method, which is
357    // less than the current maxFlushedSeqId. And if next time we only flush the family with this
358    // unusual lowestUnflushedSeqId, the maxFlushedSeqId will go backwards.
359    // This is an unexpected behavior so we should fix it, otherwise it may cause unexpected
360    // behavior in other area.
361    // The solution here is a bit hack but fine. Just replace the lowestUnflushedSeqId with
362    // maxFlushedSeqId + 1 if it is lesser. The meaning of maxFlushedSeqId is that, all edits less
363    // than or equal to it have been flushed, i.e, persistent to HFile, so set
364    // lowestUnflushedSequenceId to maxFlushedSeqId + 1 will not cause data loss.
365    // And technically, using +1 is fine here. If the maxFlushesSeqId is just the flushOpSeqId, it
366    // means we have flushed all the stores so the seq id for actual data should be at least plus 1.
367    // And if we do not flush all the stores, then the maxFlushedSeqId is calculated by
368    // lowestUnflushedSeqId - 1, so here let's plus the 1 back.
369    Long wrappedSeqId = Long.valueOf(maxFlushedSeqId + 1);
370    synchronized (tieLock) {
371      this.flushingSequenceIds.remove(encodedRegionName);
372      Map<ImmutableByteArray, Long> unflushed = lowestUnflushedSequenceIds.get(encodedRegionName);
373      if (unflushed == null) {
374        return;
375      }
376      for (Map.Entry<ImmutableByteArray, Long> e : unflushed.entrySet()) {
377        if (e.getValue().longValue() <= maxFlushedSeqId) {
378          e.setValue(wrappedSeqId);
379        }
380      }
381    }
382  }
383
384  void abortCacheFlush(final byte[] encodedRegionName) {
385    // Method is called when we are crashing down because failed write flush AND it is called
386    // if we fail prepare. The below is for the fail prepare case; we restore the old sequence ids.
387    Map<ImmutableByteArray, Long> flushing = null;
388    Map<ImmutableByteArray, Long> tmpMap = new HashMap<>();
389    // Here we are moving sequenceids from flushing back to unflushed; doing opposite of what
390    // happened in startCacheFlush. During prepare phase, we have update lock on the region so
391    // no edits should be coming in via append.
392    synchronized (tieLock) {
393      flushing = this.flushingSequenceIds.remove(encodedRegionName);
394      if (flushing != null) {
395        Map<ImmutableByteArray, Long> unflushed = getOrCreateLowestSequenceIds(encodedRegionName);
396        for (Map.Entry<ImmutableByteArray, Long> e: flushing.entrySet()) {
397          // Set into unflushed the 'old' oldest sequenceid and if any value in flushed with this
398          // value, it will now be in tmpMap.
399          tmpMap.put(e.getKey(), unflushed.put(e.getKey(), e.getValue()));
400        }
401      }
402    }
403
404    // Here we are doing some 'test' to see if edits are going in out of order. What is it for?
405    // Carried over from old code.
406    if (flushing != null) {
407      for (Map.Entry<ImmutableByteArray, Long> e : flushing.entrySet()) {
408        Long currentId = tmpMap.get(e.getKey());
409        if (currentId != null && currentId.longValue() < e.getValue().longValue()) {
410          String errorStr = Bytes.toString(encodedRegionName) + " family "
411              + e.getKey().toString() + " acquired edits out of order current memstore seq="
412              + currentId + ", previous oldest unflushed id=" + e.getValue();
413          LOG.error(errorStr);
414          Runtime.getRuntime().halt(1);
415        }
416      }
417    }
418  }
419
420  /**
421   * See if passed <code>sequenceids</code> are lower -- i.e. earlier -- than any outstanding
422   * sequenceids, sequenceids we are holding on to in this accounting instance.
423   * @param sequenceids Keyed by encoded region name. Cannot be null (doesn't make sense for it to
424   *          be null).
425   * @return true if all sequenceids are lower, older than, the old sequenceids in this instance.
426   */
427  boolean areAllLower(Map<byte[], Long> sequenceids) {
428    Map<byte[], Long> flushing = null;
429    Map<byte[], Long> unflushed = null;
430    synchronized (this.tieLock) {
431      // Get a flattened -- only the oldest sequenceid -- copy of current flushing and unflushed
432      // data structures to use in tests below.
433      flushing = flattenToLowestSequenceId(this.flushingSequenceIds);
434      unflushed = flattenToLowestSequenceId(this.lowestUnflushedSequenceIds);
435    }
436    for (Map.Entry<byte[], Long> e : sequenceids.entrySet()) {
437      long oldestFlushing = Long.MAX_VALUE;
438      long oldestUnflushed = Long.MAX_VALUE;
439      if (flushing != null && flushing.containsKey(e.getKey())) {
440        oldestFlushing = flushing.get(e.getKey());
441      }
442      if (unflushed != null && unflushed.containsKey(e.getKey())) {
443        oldestUnflushed = unflushed.get(e.getKey());
444      }
445      long min = Math.min(oldestFlushing, oldestUnflushed);
446      if (min <= e.getValue()) {
447        return false;
448      }
449    }
450    return true;
451  }
452
453  /**
454   * Iterates over the given Map and compares sequence ids with corresponding entries in
455   * {@link #lowestUnflushedSequenceIds}. If a region in
456   * {@link #lowestUnflushedSequenceIds} has a sequence id less than that passed in
457   * <code>sequenceids</code> then return it.
458   * @param sequenceids Sequenceids keyed by encoded region name.
459   * @return stores of regions found in this instance with sequence ids less than those passed in.
460   */
461  Map<byte[], List<byte[]>> findLower(Map<byte[], Long> sequenceids) {
462    Map<byte[], List<byte[]>> toFlush = null;
463    // Keeping the old behavior of iterating unflushedSeqNums under oldestSeqNumsLock.
464    synchronized (tieLock) {
465      for (Map.Entry<byte[], Long> e : sequenceids.entrySet()) {
466        Map<ImmutableByteArray, Long> m = this.lowestUnflushedSequenceIds.get(e.getKey());
467        if (m == null) {
468          continue;
469        }
470        for (Map.Entry<ImmutableByteArray, Long> me : m.entrySet()) {
471          if (me.getValue() <= e.getValue()) {
472            if (toFlush == null) {
473              toFlush = new TreeMap(Bytes.BYTES_COMPARATOR);
474            }
475            toFlush.computeIfAbsent(e.getKey(), k -> new ArrayList<>())
476              .add(Bytes.toBytes(me.getKey().toString()));
477          }
478        }
479      }
480    }
481    return toFlush;
482  }
483}