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