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}