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.CollectionUtils.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.concurrent.ConcurrentHashMap;
029import java.util.concurrent.ConcurrentMap;
030import org.apache.hadoop.hbase.HConstants;
031import org.apache.hadoop.hbase.util.Bytes;
032import org.apache.hadoop.hbase.util.ImmutableByteArray;
033import org.apache.yetus.audience.InterfaceAudience;
034import org.slf4j.Logger;
035import org.slf4j.LoggerFactory;
036
037import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
038
039/**
040 * <p>
041 * Accounting of sequence ids per region and then by column family. So we can our accounting
042 * current, call startCacheFlush and then finishedCacheFlush or abortCacheFlush so this instance can
043 * keep abreast of the state of sequence id persistence. Also call update per append.
044 * </p>
045 * <p>
046 * For the implementation, we assume that all the {@code encodedRegionName} passed in is 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
054class SequenceIdAccounting {
055
056  private static final Logger LOG = LoggerFactory.getLogger(SequenceIdAccounting.class);
057  /**
058   * This lock ties all operations on {@link SequenceIdAccounting#flushingSequenceIds} and
059   * {@link #lowestUnflushedSequenceIds} Maps. {@link #lowestUnflushedSequenceIds} has the
060   * lowest outstanding sequence ids EXCEPT when flushing. When we flush, the current
061   * lowest set for the region/column family are moved (atomically because of this lock) to
062   * {@link #flushingSequenceIds}.
063   * 
064   * <p>The two Maps are tied by this locking object EXCEPT when we go to update the lowest
065   * entry; see {@link #lowestUnflushedSequenceIds}. In here is a putIfAbsent call on
066   * {@link #lowestUnflushedSequenceIds}. In this latter case, we will add this lowest
067   * sequence id if we find that there is no entry for the current column family. There will be no
068   * entry only if we just came up OR we have moved aside current set of lowest sequence ids
069   * because the current set are being flushed (by putting them into {@link #flushingSequenceIds}).
070   * This is how we pick up the next 'lowest' sequence id per region per column family to be used
071   * figuring what is in 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,
077   * the longest-lived, their 'earliest', the 'lowest' -- sequence id.
078   *
079   * <p>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
081   * here to {@link #lowestUnflushedSequenceIds} as the next lowest sequenceid.
082   *
083   * <p>If flush fails, currently server is aborted so no need to restore previous sequence ids.
084   * <p>Needs to be concurrent Maps because we use putIfAbsent updating oldest.
085   */
086  private final ConcurrentMap<byte[], ConcurrentMap<ImmutableByteArray, Long>>
087    lowestUnflushedSequenceIds = new ConcurrentHashMap<>();
088
089  /**
090   * Map of encoded region names and family names to their lowest or OLDEST sequence/edit id
091   * currently being flushed out to hfiles. Entries are moved here from
092   * {@link #lowestUnflushedSequenceIds} while the lock {@link #tieLock} is held
093   * (so movement between the Maps is atomic).
094   */
095  private final Map<byte[], Map<ImmutableByteArray, Long>> flushingSequenceIds = new HashMap<>();
096
097  /**
098   * <p>
099   * Map of region encoded names to the latest/highest region sequence id. Updated on each call to
100   * append.
101   * </p>
102   * <p>
103   * This map uses byte[] as the key, and uses reference equality. It works in our use case as we
104   * use {@link org.apache.hadoop.hbase.client.RegionInfo#getEncodedNameAsBytes()} as keys. For a
105   * given region, it always returns the same array.
106   * </p>
107   */
108  private Map<byte[], Long> highestSequenceIds = new HashMap<>();
109
110  /**
111   * Returns the lowest unflushed sequence id for the region.
112   * @param encodedRegionName
113   * @return Lowest outstanding unflushed sequenceid for <code>encodedRegionName</code>. Will
114   * return {@link HConstants#NO_SEQNUM} when none.
115   */
116  long getLowestSequenceId(final byte[] encodedRegionName) {
117    synchronized (this.tieLock) {
118      Map<?, Long> m = this.flushingSequenceIds.get(encodedRegionName);
119      long flushingLowest = m != null ? getLowestSequenceId(m) : Long.MAX_VALUE;
120      m = this.lowestUnflushedSequenceIds.get(encodedRegionName);
121      long unflushedLowest = m != null ? getLowestSequenceId(m) : HConstants.NO_SEQNUM;
122      return Math.min(flushingLowest, unflushedLowest);
123    }
124  }
125
126  /**
127   * @param encodedRegionName
128   * @param familyName
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
168   * currently older.
169   * @param encodedRegionName
170   * @param families
171   * @param sequenceid
172   * @param lowest Whether to keep running account of oldest sequence id.
173   */
174  void update(byte[] encodedRegionName, Set<byte[]> families, long sequenceid,
175      final boolean lowest) {
176    Long l = Long.valueOf(sequenceid);
177    this.highestSequenceIds.put(encodedRegionName, l);
178    if (lowest) {
179      ConcurrentMap<ImmutableByteArray, Long> m = getOrCreateLowestSequenceIds(encodedRegionName);
180      for (byte[] familyName : families) {
181        m.putIfAbsent(ImmutableByteArray.wrap(familyName), l);
182      }
183    }
184  }
185
186  /**
187   * Update the store sequence id, e.g., upon executing in-memory compaction
188   */
189  void updateStore(byte[] encodedRegionName, byte[] familyName, Long sequenceId,
190      boolean onlyIfGreater) {
191    if (sequenceId == null) {
192      return;
193    }
194    Long highest = this.highestSequenceIds.get(encodedRegionName);
195    if (highest == null || sequenceId > highest) {
196      this.highestSequenceIds.put(encodedRegionName, sequenceId);
197    }
198    ImmutableByteArray familyNameWrapper = ImmutableByteArray.wrap(familyName);
199    synchronized (this.tieLock) {
200      ConcurrentMap<ImmutableByteArray, Long> m = getOrCreateLowestSequenceIds(encodedRegionName);
201      boolean replaced = false;
202      while (!replaced) {
203        Long oldSeqId = m.get(familyNameWrapper);
204        if (oldSeqId == null) {
205          m.put(familyNameWrapper, sequenceId);
206          replaced = true;
207        } else if (onlyIfGreater) {
208          if (sequenceId > oldSeqId) {
209            replaced = m.replace(familyNameWrapper, oldSeqId, sequenceId);
210          } else {
211            return;
212          }
213        } else { // replace even if sequence id is not greater than oldSeqId
214          m.put(familyNameWrapper, sequenceId);
215          return;
216        }
217      }
218    }
219  }
220
221  @VisibleForTesting
222  ConcurrentMap<ImmutableByteArray, Long> getOrCreateLowestSequenceIds(byte[] encodedRegionName) {
223    // Intentionally, this access is done outside of this.regionSequenceIdLock. Done per append.
224    return computeIfAbsent(this.lowestUnflushedSequenceIds, encodedRegionName,
225      ConcurrentHashMap::new);
226  }
227
228  /**
229   * @param sequenceids Map to search for lowest value.
230   * @return Lowest value found in <code>sequenceids</code>.
231   */
232  private static long getLowestSequenceId(Map<?, Long> sequenceids) {
233    long lowest = HConstants.NO_SEQNUM;
234    for (Long sid: sequenceids.values()) {
235      if (lowest == HConstants.NO_SEQNUM || sid.longValue() < lowest) {
236        lowest = sid.longValue();
237      }
238    }
239    return lowest;
240  }
241
242  /**
243   * @param src
244   * @return New Map that has same keys as <code>src</code> but instead of a Map for a value, it
245   *         instead has found the smallest sequence id and it returns that as the value instead.
246   */
247  private <T extends Map<?, Long>> Map<byte[], Long> flattenToLowestSequenceId(Map<byte[], T> src) {
248    if (src == null || src.isEmpty()) {
249      return null;
250    }
251    Map<byte[], Long> tgt = new HashMap<>();
252    for (Map.Entry<byte[], T> entry : src.entrySet()) {
253      long lowestSeqId = getLowestSequenceId(entry.getValue());
254      if (lowestSeqId != HConstants.NO_SEQNUM) {
255        tgt.put(entry.getKey(), lowestSeqId);
256      }
257    }
258    return tgt;
259  }
260
261  /**
262   * @param encodedRegionName Region to flush.
263   * @param families Families to flush. May be a subset of all families in the region.
264   * @return Returns {@link HConstants#NO_SEQNUM} if we are flushing the whole region OR if
265   * we are flushing a subset of all families but there are no edits in those families not
266   * being flushed; in other words, this is effectively same as a flush of all of the region
267   * though we were passed a subset of regions. Otherwise, it returns the sequence id of the
268   * oldest/lowest outstanding edit.
269   */
270  Long startCacheFlush(final byte[] encodedRegionName, final Set<byte[]> families) {
271    Map<byte[],Long> familytoSeq = new HashMap<>();
272    for (byte[] familyName : families){
273      familytoSeq.put(familyName,HConstants.NO_SEQNUM);
274    }
275    return startCacheFlush(encodedRegionName,familytoSeq);
276  }
277
278  Long startCacheFlush(final byte[] encodedRegionName, final Map<byte[], Long> familyToSeq) {
279    Map<ImmutableByteArray, Long> oldSequenceIds = null;
280    Long lowestUnflushedInRegion = HConstants.NO_SEQNUM;
281    synchronized (tieLock) {
282      Map<ImmutableByteArray, Long> m = this.lowestUnflushedSequenceIds.get(encodedRegionName);
283      if (m != null) {
284        // NOTE: Removal from this.lowestUnflushedSequenceIds must be done in controlled
285        // circumstance because another concurrent thread now may add sequenceids for this family
286        // (see above in getOrCreateLowestSequenceId). Make sure you are ok with this. Usually it
287        // is fine because updates are blocked when this method is called. Make sure!!!
288        for (Map.Entry<byte[], Long> entry : familyToSeq.entrySet()) {
289          ImmutableByteArray familyNameWrapper = ImmutableByteArray.wrap((byte[]) entry.getKey());
290          Long seqId = null;
291          if(entry.getValue() == HConstants.NO_SEQNUM) {
292            seqId = m.remove(familyNameWrapper);
293          } else {
294            seqId = m.replace(familyNameWrapper, entry.getValue());
295          }
296          if (seqId != null) {
297            if (oldSequenceIds == null) {
298              oldSequenceIds = new HashMap<>();
299            }
300            oldSequenceIds.put(familyNameWrapper, seqId);
301          }
302        }
303        if (oldSequenceIds != null && !oldSequenceIds.isEmpty()) {
304          if (this.flushingSequenceIds.put(encodedRegionName, oldSequenceIds) != null) {
305            LOG.warn("Flushing Map not cleaned up for " + Bytes.toString(encodedRegionName) +
306              ", sequenceid=" + oldSequenceIds);
307          }
308        }
309        if (m.isEmpty()) {
310          // Remove it otherwise it will be in oldestUnflushedStoreSequenceIds for ever
311          // even if the region is already moved to other server.
312          // Do not worry about data racing, we held write lock of region when calling
313          // startCacheFlush, so no one can add value to the map we removed.
314          this.lowestUnflushedSequenceIds.remove(encodedRegionName);
315        } else {
316          // Flushing a subset of the region families. Return the sequence id of the oldest entry.
317          lowestUnflushedInRegion = Collections.min(m.values());
318        }
319      }
320    }
321    // Do this check outside lock.
322    if (oldSequenceIds != null && oldSequenceIds.isEmpty()) {
323      // TODO: if we have no oldStoreSeqNum, and WAL is not disabled, presumably either
324      // the region is already flushing (which would make this call invalid), or there
325      // were no appends after last flush, so why are we starting flush? Maybe we should
326      // assert not empty. Less rigorous, but safer, alternative is telling the caller to stop.
327      // For now preserve old logic.
328      LOG.warn("Couldn't find oldest sequenceid for " + Bytes.toString(encodedRegionName));
329    }
330    return lowestUnflushedInRegion;
331  }
332
333  void completeCacheFlush(final byte[] encodedRegionName) {
334    synchronized (tieLock) {
335      this.flushingSequenceIds.remove(encodedRegionName);
336    }
337  }
338
339  void abortCacheFlush(final byte[] encodedRegionName) {
340    // Method is called when we are crashing down because failed write flush AND it is called
341    // if we fail prepare. The below is for the fail prepare case; we restore the old sequence ids.
342    Map<ImmutableByteArray, Long> flushing = null;
343    Map<ImmutableByteArray, Long> tmpMap = new HashMap<>();
344    // Here we are moving sequenceids from flushing back to unflushed; doing opposite of what
345    // happened in startCacheFlush. During prepare phase, we have update lock on the region so
346    // no edits should be coming in via append.
347    synchronized (tieLock) {
348      flushing = this.flushingSequenceIds.remove(encodedRegionName);
349      if (flushing != null) {
350        Map<ImmutableByteArray, Long> unflushed = getOrCreateLowestSequenceIds(encodedRegionName);
351        for (Map.Entry<ImmutableByteArray, Long> e: flushing.entrySet()) {
352          // Set into unflushed the 'old' oldest sequenceid and if any value in flushed with this
353          // value, it will now be in tmpMap.
354          tmpMap.put(e.getKey(), unflushed.put(e.getKey(), e.getValue()));
355        }
356      }
357    }
358
359    // Here we are doing some 'test' to see if edits are going in out of order. What is it for?
360    // Carried over from old code.
361    if (flushing != null) {
362      for (Map.Entry<ImmutableByteArray, Long> e : flushing.entrySet()) {
363        Long currentId = tmpMap.get(e.getKey());
364        if (currentId != null && currentId.longValue() < e.getValue().longValue()) {
365          String errorStr = Bytes.toString(encodedRegionName) + " family "
366              + e.getKey().toStringUtf8() + " acquired edits out of order current memstore seq="
367              + currentId + ", previous oldest unflushed id=" + e.getValue();
368          LOG.error(errorStr);
369          Runtime.getRuntime().halt(1);
370        }
371      }
372    }
373  }
374
375  /**
376   * See if passed <code>sequenceids</code> are lower -- i.e. earlier -- than any outstanding
377   * sequenceids, sequenceids we are holding on to in this accounting instance.
378   * @param sequenceids Keyed by encoded region name. Cannot be null (doesn't make sense for it to
379   *          be null).
380   * @return true if all sequenceids are lower, older than, the old sequenceids in this instance.
381   */
382  boolean areAllLower(Map<byte[], Long> sequenceids) {
383    Map<byte[], Long> flushing = null;
384    Map<byte[], Long> unflushed = null;
385    synchronized (this.tieLock) {
386      // Get a flattened -- only the oldest sequenceid -- copy of current flushing and unflushed
387      // data structures to use in tests below.
388      flushing = flattenToLowestSequenceId(this.flushingSequenceIds);
389      unflushed = flattenToLowestSequenceId(this.lowestUnflushedSequenceIds);
390    }
391    for (Map.Entry<byte[], Long> e : sequenceids.entrySet()) {
392      long oldestFlushing = Long.MAX_VALUE;
393      long oldestUnflushed = Long.MAX_VALUE;
394      if (flushing != null && flushing.containsKey(e.getKey())) {
395        oldestFlushing = flushing.get(e.getKey());
396      }
397      if (unflushed != null && unflushed.containsKey(e.getKey())) {
398        oldestUnflushed = unflushed.get(e.getKey());
399      }
400      long min = Math.min(oldestFlushing, oldestUnflushed);
401      if (min <= e.getValue()) {
402        return false;
403      }
404    }
405    return true;
406  }
407
408  /**
409   * Iterates over the given Map and compares sequence ids with corresponding entries in
410   * {@link #lowestUnflushedSequenceIds}. If a region in
411   * {@link #lowestUnflushedSequenceIds} has a sequence id less than that passed in
412   * <code>sequenceids</code> then return it.
413   * @param sequenceids Sequenceids keyed by encoded region name.
414   * @return regions found in this instance with sequence ids less than those passed in.
415   */
416  byte[][] findLower(Map<byte[], Long> sequenceids) {
417    List<byte[]> toFlush = null;
418    // Keeping the old behavior of iterating unflushedSeqNums under oldestSeqNumsLock.
419    synchronized (tieLock) {
420      for (Map.Entry<byte[], Long> e : sequenceids.entrySet()) {
421        Map<ImmutableByteArray, Long> m = this.lowestUnflushedSequenceIds.get(e.getKey());
422        if (m == null) {
423          continue;
424        }
425        // The lowest sequence id outstanding for this region.
426        long lowest = getLowestSequenceId(m);
427        if (lowest != HConstants.NO_SEQNUM && lowest <= e.getValue()) {
428          if (toFlush == null) {
429            toFlush = new ArrayList<>();
430          }
431          toFlush.add(e.getKey());
432        }
433      }
434    }
435    return toFlush == null ? null : toFlush.toArray(new byte[0][]);
436  }
437}