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.regionserver.wal.WALActionsListener.RollRequestReason.ERROR; 021import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRequestReason.LOW_REPLICATION; 022import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRequestReason.SIZE; 023import static org.apache.hadoop.hbase.regionserver.wal.WALActionsListener.RollRequestReason.SLOW_SYNC; 024 025import com.lmax.disruptor.BlockingWaitStrategy; 026import com.lmax.disruptor.EventHandler; 027import com.lmax.disruptor.ExceptionHandler; 028import com.lmax.disruptor.LifecycleAware; 029import com.lmax.disruptor.TimeoutException; 030import com.lmax.disruptor.dsl.Disruptor; 031import com.lmax.disruptor.dsl.ProducerType; 032import io.opentelemetry.api.trace.Span; 033import java.io.IOException; 034import java.io.OutputStream; 035import java.util.Arrays; 036import java.util.List; 037import java.util.concurrent.BlockingQueue; 038import java.util.concurrent.CountDownLatch; 039import java.util.concurrent.ExecutorService; 040import java.util.concurrent.Executors; 041import java.util.concurrent.LinkedBlockingQueue; 042import java.util.concurrent.TimeUnit; 043import java.util.concurrent.atomic.AtomicInteger; 044import org.apache.hadoop.conf.Configuration; 045import org.apache.hadoop.fs.FSDataOutputStream; 046import org.apache.hadoop.fs.FileSystem; 047import org.apache.hadoop.fs.Path; 048import org.apache.hadoop.hbase.Abortable; 049import org.apache.hadoop.hbase.HConstants; 050import org.apache.hadoop.hbase.client.RegionInfo; 051import org.apache.hadoop.hbase.util.Bytes; 052import org.apache.hadoop.hbase.util.ClassSize; 053import org.apache.hadoop.hbase.util.CommonFSUtils; 054import org.apache.hadoop.hbase.util.Threads; 055import org.apache.hadoop.hbase.wal.FSHLogProvider; 056import org.apache.hadoop.hbase.wal.WALEdit; 057import org.apache.hadoop.hbase.wal.WALKeyImpl; 058import org.apache.hadoop.hbase.wal.WALProvider.Writer; 059import org.apache.hadoop.hdfs.DFSOutputStream; 060import org.apache.hadoop.hdfs.client.HdfsDataOutputStream; 061import org.apache.hadoop.hdfs.protocol.DatanodeInfo; 062import org.apache.yetus.audience.InterfaceAudience; 063import org.slf4j.Logger; 064import org.slf4j.LoggerFactory; 065 066import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; 067 068/** 069 * The original implementation of FSWAL. 070 */ 071@InterfaceAudience.Private 072public class FSHLog extends AbstractFSWAL<Writer> { 073 // IMPLEMENTATION NOTES: 074 // 075 // At the core is a ring buffer. Our ring buffer is the LMAX Disruptor. It tries to 076 // minimize synchronizations and volatile writes when multiple contending threads as is the case 077 // here appending and syncing on a single WAL. The Disruptor is configured to handle multiple 078 // producers but it has one consumer only (the producers in HBase are IPC Handlers calling append 079 // and then sync). The single consumer/writer pulls the appends and syncs off the ring buffer. 080 // When a handler calls sync, it is given back a future. The producer 'blocks' on the future so 081 // it does not return until the sync completes. The future is passed over the ring buffer from 082 // the producer/handler to the consumer thread where it does its best to batch up the producer 083 // syncs so one WAL sync actually spans multiple producer sync invocations. How well the 084 // batching works depends on the write rate; i.e. we tend to batch more in times of 085 // high writes/syncs. 086 // 087 // Calls to append now also wait until the append has been done on the consumer side of the 088 // disruptor. We used to not wait but it makes the implementation easier to grok if we have 089 // the region edit/sequence id after the append returns. 090 // 091 // TODO: Handlers need to coordinate appending AND syncing. Can we have the threads contend 092 // once only? Probably hard given syncs take way longer than an append. 093 // 094 // The consumer threads pass the syncs off to multiple syncing threads in a round robin fashion 095 // to ensure we keep up back-to-back FS sync calls (FS sync calls are the long poll writing the 096 // WAL). The consumer thread passes the futures to the sync threads for it to complete 097 // the futures when done. 098 // 099 // The 'sequence' in the below is the sequence of the append/sync on the ringbuffer. It 100 // acts as a sort-of transaction id. It is always incrementing. 101 // 102 // The RingBufferEventHandler class hosts the ring buffer consuming code. The threads that 103 // do the actual FS sync are implementations of SyncRunner. SafePointZigZagLatch is a 104 // synchronization class used to halt the consumer at a safe point -- just after all outstanding 105 // syncs and appends have completed -- so the log roller can swap the WAL out under it. 106 // 107 // We use ring buffer sequence as txid of FSWALEntry and SyncFuture. 108 private static final Logger LOG = LoggerFactory.getLogger(FSHLog.class); 109 110 private static final String TOLERABLE_LOW_REPLICATION = 111 "hbase.regionserver.hlog.tolerable.lowreplication"; 112 private static final String LOW_REPLICATION_ROLL_LIMIT = 113 "hbase.regionserver.hlog.lowreplication.rolllimit"; 114 private static final int DEFAULT_LOW_REPLICATION_ROLL_LIMIT = 5; 115 private static final String ROLL_ERRORS_TOLERATED = "hbase.regionserver.logroll.errors.tolerated"; 116 private static final int DEFAULT_ROLL_ERRORS_TOLERATED = 2; 117 private static final String SYNCER_COUNT = "hbase.regionserver.hlog.syncer.count"; 118 private static final int DEFAULT_SYNCER_COUNT = 5; 119 private static final String MAX_BATCH_COUNT = "hbase.regionserver.wal.sync.batch.count"; 120 private static final int DEFAULT_MAX_BATCH_COUNT = 200; 121 122 private static final String FSHLOG_WAIT_ON_SHUTDOWN_IN_SECONDS = 123 "hbase.wal.fshlog.wait.on.shutdown.seconds"; 124 private static final int DEFAULT_FSHLOG_WAIT_ON_SHUTDOWN_IN_SECONDS = 5; 125 126 /** 127 * The nexus at which all incoming handlers meet. Does appends and sync with an ordering. Appends 128 * and syncs are each put on the ring which means handlers need to smash up against the ring twice 129 * (can we make it once only? ... maybe not since time to append is so different from time to sync 130 * and sometimes we don't want to sync or we want to async the sync). The ring is where we make 131 * sure of our ordering and it is also where we do batching up of handler sync calls. 132 */ 133 private final Disruptor<RingBufferTruck> disruptor; 134 135 /** 136 * This fellow is run by the above appendExecutor service but it is all about batching up appends 137 * and syncs; it may shutdown without cleaning out the last few appends or syncs. To guard against 138 * this, keep a reference to this handler and do explicit close on way out to make sure all 139 * flushed out before we exit. 140 */ 141 private final RingBufferEventHandler ringBufferEventHandler; 142 143 /** 144 * FSDataOutputStream associated with the current SequenceFile.writer 145 */ 146 private FSDataOutputStream hdfs_out; 147 148 // All about log rolling if not enough replicas outstanding. 149 150 // Minimum tolerable replicas, if the actual value is lower than it, rollWriter will be triggered 151 private final int minTolerableReplication; 152 153 // If live datanode count is lower than the default replicas value, 154 // RollWriter will be triggered in each sync(So the RollWriter will be 155 // triggered one by one in a short time). Using it as a workaround to slow 156 // down the roll frequency triggered by checkLowReplication(). 157 private final AtomicInteger consecutiveLogRolls = new AtomicInteger(0); 158 159 private final int lowReplicationRollLimit; 160 161 // If consecutiveLogRolls is larger than lowReplicationRollLimit, 162 // then disable the rolling in checkLowReplication(). 163 // Enable it if the replications recover. 164 private volatile boolean lowReplicationRollEnabled = true; 165 166 /** Number of log close errors tolerated before we abort */ 167 private final int closeErrorsTolerated; 168 169 private final AtomicInteger closeErrorCount = new AtomicInteger(); 170 171 private final int waitOnShutdownInSeconds; 172 private final ExecutorService closeExecutor = Executors.newCachedThreadPool( 173 new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Close-WAL-Writer-%d").build()); 174 175 /** 176 * Exception handler to pass the disruptor ringbuffer. Same as native implementation only it logs 177 * using our logger instead of java native logger. 178 */ 179 static class RingBufferExceptionHandler implements ExceptionHandler<RingBufferTruck> { 180 181 @Override 182 public void handleEventException(Throwable ex, long sequence, RingBufferTruck event) { 183 LOG.error("Sequence=" + sequence + ", event=" + event, ex); 184 throw new RuntimeException(ex); 185 } 186 187 @Override 188 public void handleOnStartException(Throwable ex) { 189 LOG.error(ex.toString(), ex); 190 throw new RuntimeException(ex); 191 } 192 193 @Override 194 public void handleOnShutdownException(Throwable ex) { 195 LOG.error(ex.toString(), ex); 196 throw new RuntimeException(ex); 197 } 198 } 199 200 /** 201 * Constructor. 202 * @param fs filesystem handle 203 * @param root path for stored and archived wals 204 * @param logDir dir where wals are stored 205 * @param conf configuration to use 206 */ 207 public FSHLog(final FileSystem fs, final Path root, final String logDir, final Configuration conf) 208 throws IOException { 209 this(fs, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, null); 210 } 211 212 public FSHLog(final FileSystem fs, Abortable abortable, final Path root, final String logDir, 213 final Configuration conf) throws IOException { 214 this(fs, abortable, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, 215 null); 216 } 217 218 public FSHLog(final FileSystem fs, final Path rootDir, final String logDir, 219 final String archiveDir, final Configuration conf, final List<WALActionsListener> listeners, 220 final boolean failIfWALExists, final String prefix, final String suffix) throws IOException { 221 this(fs, null, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix); 222 } 223 224 /** 225 * Create an edit log at the given <code>dir</code> location. You should never have to load an 226 * existing log. If there is a log at startup, it should have already been processed and deleted 227 * by the time the WAL object is started up. 228 * @param fs filesystem handle 229 * @param abortable Abortable - the server here 230 * @param rootDir path to where logs and oldlogs 231 * @param logDir dir where wals are stored 232 * @param archiveDir dir where wals are archived 233 * @param conf configuration to use 234 * @param listeners Listeners on WAL events. Listeners passed here will be registered before 235 * we do anything else; e.g. the Constructor {@link #rollWriter()}. 236 * @param failIfWALExists If true IOException will be thrown if files related to this wal already 237 * exist. 238 * @param prefix should always be hostname and port in distributed env and it will be URL 239 * encoded before being used. If prefix is null, "wal" will be used 240 * @param suffix will be url encoded. null is treated as empty. non-empty must start with 241 * {@link org.apache.hadoop.hbase.wal.AbstractFSWALProvider#WAL_FILE_NAME_DELIMITER} 242 */ 243 public FSHLog(final FileSystem fs, final Abortable abortable, final Path rootDir, 244 final String logDir, final String archiveDir, final Configuration conf, 245 final List<WALActionsListener> listeners, final boolean failIfWALExists, final String prefix, 246 final String suffix) throws IOException { 247 super(fs, abortable, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, 248 suffix); 249 this.minTolerableReplication = 250 conf.getInt(TOLERABLE_LOW_REPLICATION, CommonFSUtils.getDefaultReplication(fs, this.walDir)); 251 this.lowReplicationRollLimit = 252 conf.getInt(LOW_REPLICATION_ROLL_LIMIT, DEFAULT_LOW_REPLICATION_ROLL_LIMIT); 253 this.closeErrorsTolerated = conf.getInt(ROLL_ERRORS_TOLERATED, DEFAULT_ROLL_ERRORS_TOLERATED); 254 this.waitOnShutdownInSeconds = 255 conf.getInt(FSHLOG_WAIT_ON_SHUTDOWN_IN_SECONDS, DEFAULT_FSHLOG_WAIT_ON_SHUTDOWN_IN_SECONDS); 256 // This is the 'writer' -- a single threaded executor. This single thread 'consumes' what is 257 // put on the ring buffer. 258 String hostingThreadName = Thread.currentThread().getName(); 259 // Using BlockingWaitStrategy. Stuff that is going on here takes so long it makes no sense 260 // spinning as other strategies do. 261 this.disruptor = new Disruptor<>(RingBufferTruck::new, getPreallocatedEventCount(), 262 new ThreadFactoryBuilder().setNameFormat(hostingThreadName + ".append-pool-%d") 263 .setDaemon(true).setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build(), 264 ProducerType.MULTI, new BlockingWaitStrategy()); 265 // Advance the ring buffer sequence so that it starts from 1 instead of 0, 266 // because SyncFuture.NOT_DONE = 0. 267 this.disruptor.getRingBuffer().next(); 268 int syncerCount = conf.getInt(SYNCER_COUNT, DEFAULT_SYNCER_COUNT); 269 int maxBatchCount = conf.getInt(MAX_BATCH_COUNT, 270 conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT, DEFAULT_MAX_BATCH_COUNT)); 271 this.ringBufferEventHandler = new RingBufferEventHandler(syncerCount, maxBatchCount); 272 this.disruptor.setDefaultExceptionHandler(new RingBufferExceptionHandler()); 273 this.disruptor.handleEventsWith(new RingBufferEventHandler[] { this.ringBufferEventHandler }); 274 // Starting up threads in constructor is a no no; Interface should have an init call. 275 this.disruptor.start(); 276 } 277 278 /** 279 * Currently, we need to expose the writer's OutputStream to tests so that they can manipulate the 280 * default behavior (such as setting the maxRecoveryErrorCount value). This is done using 281 * reflection on the underlying HDFS OutputStream. NOTE: This could be removed once Hadoop1 282 * support is removed. 283 * @return null if underlying stream is not ready. 284 */ 285 OutputStream getOutputStream() { 286 FSDataOutputStream fsdos = this.hdfs_out; 287 return fsdos != null ? fsdos.getWrappedStream() : null; 288 } 289 290 /** 291 * Run a sync after opening to set up the pipeline. 292 */ 293 private void preemptiveSync(final ProtobufLogWriter nextWriter) { 294 long startTimeNanos = System.nanoTime(); 295 try { 296 nextWriter.sync(useHsync); 297 postSync(System.nanoTime() - startTimeNanos, 0); 298 } catch (IOException e) { 299 // optimization failed, no need to abort here. 300 LOG.warn("pre-sync failed but an optimization so keep going", e); 301 } 302 } 303 304 /** 305 * This method allows subclasses to inject different writers without having to extend other 306 * methods like rollWriter(). 307 * @return Writer instance 308 */ 309 @Override 310 protected Writer createWriterInstance(final Path path) throws IOException { 311 Writer writer = FSHLogProvider.createWriter(conf, fs, path, false, this.blocksize); 312 if (writer instanceof ProtobufLogWriter) { 313 preemptiveSync((ProtobufLogWriter) writer); 314 } 315 return writer; 316 } 317 318 /** 319 * Used to manufacture race condition reliably. For testing only. 320 * @see #beforeWaitOnSafePoint() 321 */ 322 protected void afterCreatingZigZagLatch() { 323 } 324 325 /** 326 * @see #afterCreatingZigZagLatch() 327 */ 328 protected void beforeWaitOnSafePoint() { 329 } 330 331 @Override 332 protected void doAppend(Writer writer, FSWALEntry entry) throws IOException { 333 writer.append(entry); 334 } 335 336 @Override 337 protected void doReplaceWriter(Path oldPath, Path newPath, Writer nextWriter) throws IOException { 338 // Ask the ring buffer writer to pause at a safe point. Once we do this, the writer 339 // thread will eventually pause. An error hereafter needs to release the writer thread 340 // regardless -- hence the finally block below. Note, this method is called from the FSHLog 341 // constructor BEFORE the ring buffer is set running so it is null on first time through 342 // here; allow for that. 343 SyncFuture syncFuture = null; 344 SafePointZigZagLatch zigzagLatch = null; 345 long sequence = -1L; 346 if (this.writer != null && this.ringBufferEventHandler != null) { 347 // Get sequence first to avoid dead lock when ring buffer is full 348 // Considering below sequence 349 // 1. replaceWriter is called and zigzagLatch is initialized 350 // 2. ringBufferEventHandler#onEvent is called and arrives at #attainSafePoint(long) then wait 351 // on safePointReleasedLatch 352 // 3. Since ring buffer is full, if we get sequence when publish sync, the replaceWriter 353 // thread will wait for the ring buffer to be consumed, but the only consumer is waiting 354 // replaceWriter thread to release safePointReleasedLatch, which causes a deadlock 355 sequence = getSequenceOnRingBuffer(); 356 zigzagLatch = this.ringBufferEventHandler.attainSafePoint(); 357 } 358 afterCreatingZigZagLatch(); 359 try { 360 // Wait on the safe point to be achieved. Send in a sync in case nothing has hit the 361 // ring buffer between the above notification of writer that we want it to go to 362 // 'safe point' and then here where we are waiting on it to attain safe point. Use 363 // 'sendSync' instead of 'sync' because we do not want this thread to block waiting on it 364 // to come back. Cleanup this syncFuture down below after we are ready to run again. 365 try { 366 if (zigzagLatch != null) { 367 // use assert to make sure no change breaks the logic that 368 // sequence and zigzagLatch will be set together 369 assert sequence > 0L : "Failed to get sequence from ring buffer"; 370 syncFuture = zigzagLatch.waitSafePoint(publishSyncOnRingBuffer(sequence, false)); 371 } 372 } catch (FailedSyncBeforeLogCloseException e) { 373 // If unflushed/unsynced entries on close, it is reason to abort. 374 if (isUnflushedEntries()) { 375 throw e; 376 } 377 LOG.warn( 378 "Failed sync-before-close but no outstanding appends; closing WAL" + e.getMessage()); 379 } 380 long oldFileLen = 0L; 381 // It is at the safe point. Swap out writer from under the blocked writer thread. 382 if (this.writer != null) { 383 oldFileLen = this.writer.getLength(); 384 // In case of having unflushed entries or we already reached the 385 // closeErrorsTolerated count, call the closeWriter inline rather than in async 386 // way so that in case of an IOE we will throw it back and abort RS. 387 inflightWALClosures.put(oldPath.getName(), writer); 388 if (isUnflushedEntries() || closeErrorCount.get() >= this.closeErrorsTolerated) { 389 closeWriter(this.writer, oldPath, true); 390 } else { 391 Writer localWriter = this.writer; 392 closeExecutor.execute(() -> { 393 try { 394 closeWriter(localWriter, oldPath, false); 395 } catch (IOException e) { 396 // We will never reach here. 397 } 398 }); 399 } 400 } 401 logRollAndSetupWalProps(oldPath, newPath, oldFileLen); 402 this.writer = nextWriter; 403 if (nextWriter != null && nextWriter instanceof ProtobufLogWriter) { 404 this.hdfs_out = ((ProtobufLogWriter) nextWriter).getStream(); 405 } else { 406 this.hdfs_out = null; 407 } 408 } catch (InterruptedException ie) { 409 // Perpetuate the interrupt 410 Thread.currentThread().interrupt(); 411 } catch (IOException e) { 412 long count = getUnflushedEntriesCount(); 413 LOG.error("Failed close of WAL writer " + oldPath + ", unflushedEntries=" + count, e); 414 throw new FailedLogCloseException(oldPath + ", unflushedEntries=" + count, e); 415 } finally { 416 // Let the writer thread go regardless, whether error or not. 417 if (zigzagLatch != null) { 418 // Reset rollRequested status 419 rollRequested.set(false); 420 zigzagLatch.releaseSafePoint(); 421 // syncFuture will be null if we failed our wait on safe point above. Otherwise, if 422 // latch was obtained successfully, the sync we threw in either trigger the latch or it 423 // got stamped with an exception because the WAL was damaged and we could not sync. Now 424 // the write pipeline has been opened up again by releasing the safe point, process the 425 // syncFuture we got above. This is probably a noop but it may be stale exception from 426 // when old WAL was in place. Catch it if so. 427 if (syncFuture != null) { 428 try { 429 blockOnSync(syncFuture); 430 } catch (IOException ioe) { 431 if (LOG.isTraceEnabled()) { 432 LOG.trace("Stale sync exception", ioe); 433 } 434 } 435 } 436 } 437 } 438 } 439 440 private void closeWriter(Writer writer, Path path, boolean syncCloseCall) throws IOException { 441 Span span = Span.current(); 442 try { 443 span.addEvent("closing writer"); 444 writer.close(); 445 span.addEvent("writer closed"); 446 } catch (IOException ioe) { 447 int errors = closeErrorCount.incrementAndGet(); 448 boolean hasUnflushedEntries = isUnflushedEntries(); 449 if (syncCloseCall && (hasUnflushedEntries || (errors > this.closeErrorsTolerated))) { 450 LOG.error("Close of WAL " + path + " failed. Cause=\"" + ioe.getMessage() + "\", errors=" 451 + errors + ", hasUnflushedEntries=" + hasUnflushedEntries); 452 throw ioe; 453 } 454 LOG.warn("Riding over failed WAL close of " + path 455 + "; THIS FILE WAS NOT CLOSED BUT ALL EDITS SYNCED SO SHOULD BE OK", ioe); 456 } finally { 457 inflightWALClosures.remove(path.getName()); 458 } 459 } 460 461 @Override 462 protected void doShutdown() throws IOException { 463 // Shutdown the disruptor. Will stop after all entries have been processed. Make sure we 464 // have stopped incoming appends before calling this else it will not shutdown. We are 465 // conservative below waiting a long time and if not elapsed, then halting. 466 if (this.disruptor != null) { 467 long timeoutms = conf.getLong("hbase.wal.disruptor.shutdown.timeout.ms", 60000); 468 try { 469 this.disruptor.shutdown(timeoutms, TimeUnit.MILLISECONDS); 470 } catch (TimeoutException e) { 471 LOG.warn("Timed out bringing down disruptor after " + timeoutms + "ms; forcing halt " 472 + "(It is a problem if this is NOT an ABORT! -- DATALOSS!!!!)"); 473 this.disruptor.halt(); 474 this.disruptor.shutdown(); 475 } 476 } 477 478 if (LOG.isDebugEnabled()) { 479 LOG.debug("Closing WAL writer in " + CommonFSUtils.getPath(walDir)); 480 } 481 if (this.writer != null) { 482 this.writer.close(); 483 this.writer = null; 484 } 485 closeExecutor.shutdown(); 486 try { 487 if (!closeExecutor.awaitTermination(waitOnShutdownInSeconds, TimeUnit.SECONDS)) { 488 LOG.error( 489 "We have waited {} seconds but the close of writer(s) doesn't complete." 490 + "Please check the status of underlying filesystem" 491 + " or increase the wait time by the config \"{}\"", 492 this.waitOnShutdownInSeconds, FSHLOG_WAIT_ON_SHUTDOWN_IN_SECONDS); 493 } 494 } catch (InterruptedException e) { 495 LOG.error("The wait for termination of FSHLog writer(s) is interrupted"); 496 Thread.currentThread().interrupt(); 497 } 498 } 499 500 @Override 501 protected long append(final RegionInfo hri, final WALKeyImpl key, final WALEdit edits, 502 final boolean inMemstore) throws IOException { 503 return stampSequenceIdAndPublishToRingBuffer(hri, key, edits, inMemstore, 504 disruptor.getRingBuffer()); 505 } 506 507 /** 508 * Thread to runs the hdfs sync call. This call takes a while to complete. This is the longest 509 * pole adding edits to the WAL and this must complete to be sure all edits persisted. We run 510 * multiple threads sync'ng rather than one that just syncs in series so we have better latencies; 511 * otherwise, an edit that arrived just after a sync started, might have to wait almost the length 512 * of two sync invocations before it is marked done. 513 * <p> 514 * When the sync completes, it marks all the passed in futures done. On the other end of the sync 515 * future is a blocked thread, usually a regionserver Handler. There may be more than one future 516 * passed in the case where a few threads arrive at about the same time and all invoke 'sync'. In 517 * this case we'll batch up the invocations and run one filesystem sync only for a batch of 518 * Handler sync invocations. Do not confuse these Handler SyncFutures with the futures an 519 * ExecutorService returns when you call submit. We have no use for these in this model. These 520 * SyncFutures are 'artificial', something to hold the Handler until the filesystem sync 521 * completes. 522 */ 523 private class SyncRunner extends Thread { 524 private volatile long sequence; 525 // Keep around last exception thrown. Clear on successful sync. 526 private final BlockingQueue<SyncFuture> syncFutures; 527 private volatile SyncFuture takeSyncFuture = null; 528 529 SyncRunner(final String name, final int maxHandlersCount) { 530 super(name); 531 // LinkedBlockingQueue because of 532 // http://www.javacodegeeks.com/2010/09/java-best-practices-queue-battle-and.html 533 // Could use other blockingqueues here or concurrent queues. 534 // 535 // We could let the capacity be 'open' but bound it so we get alerted in pathological case 536 // where we cannot sync and we have a bunch of threads all backed up waiting on their syncs 537 // to come in. LinkedBlockingQueue actually shrinks when you remove elements so Q should 538 // stay neat and tidy in usual case. Let the max size be three times the maximum handlers. 539 // The passed in maxHandlerCount is the user-level handlers which is what we put up most of 540 // but HBase has other handlers running too -- opening region handlers which want to write 541 // the meta table when succesful (i.e. sync), closing handlers -- etc. These are usually 542 // much fewer in number than the user-space handlers so Q-size should be user handlers plus 543 // some space for these other handlers. Lets multiply by 3 for good-measure. 544 this.syncFutures = new LinkedBlockingQueue<>(maxHandlersCount * 3); 545 } 546 547 void offer(final long sequence, final SyncFuture[] syncFutures, final int syncFutureCount) { 548 // Set sequence first because the add to the queue will wake the thread if sleeping. 549 this.sequence = sequence; 550 for (int i = 0; i < syncFutureCount; ++i) { 551 this.syncFutures.add(syncFutures[i]); 552 } 553 } 554 555 /** 556 * Release the passed <code>syncFuture</code> 557 * @return Returns 1. 558 */ 559 private int releaseSyncFuture(final SyncFuture syncFuture, final long currentSequence, 560 final Throwable t) { 561 if (!syncFuture.done(currentSequence, t)) { 562 throw new IllegalStateException(); 563 } 564 565 // This function releases one sync future only. 566 return 1; 567 } 568 569 /** 570 * Release all SyncFutures whose sequence is <= <code>currentSequence</code>. 571 * @param t May be non-null if we are processing SyncFutures because an exception was thrown. 572 * @return Count of SyncFutures we let go. 573 */ 574 private int releaseSyncFutures(final long currentSequence, final Throwable t) { 575 int syncCount = 0; 576 for (SyncFuture syncFuture; (syncFuture = this.syncFutures.peek()) != null;) { 577 if (syncFuture.getTxid() > currentSequence) { 578 break; 579 } 580 releaseSyncFuture(syncFuture, currentSequence, t); 581 if (!this.syncFutures.remove(syncFuture)) { 582 throw new IllegalStateException(syncFuture.toString()); 583 } 584 syncCount++; 585 } 586 return syncCount; 587 } 588 589 /** 590 * @param sequence The sequence we ran the filesystem sync against. 591 * @return Current highest synced sequence. 592 */ 593 private long updateHighestSyncedSequence(long sequence) { 594 long currentHighestSyncedSequence; 595 // Set the highestSyncedSequence IFF our current sequence id is the 'highest'. 596 do { 597 currentHighestSyncedSequence = highestSyncedTxid.get(); 598 if (currentHighestSyncedSequence >= sequence) { 599 // Set the sync number to current highwater mark; might be able to let go more 600 // queued sync futures 601 sequence = currentHighestSyncedSequence; 602 break; 603 } 604 } while (!highestSyncedTxid.compareAndSet(currentHighestSyncedSequence, sequence)); 605 return sequence; 606 } 607 608 boolean areSyncFuturesReleased() { 609 // check whether there is no sync futures offered, and no in-flight sync futures that is being 610 // processed. 611 return syncFutures.size() <= 0 && takeSyncFuture == null; 612 } 613 614 @Override 615 public void run() { 616 long currentSequence; 617 while (!isInterrupted()) { 618 int syncCount = 0; 619 620 try { 621 // Make a local copy of takeSyncFuture after we get it. We've been running into NPEs 622 // 2020-03-22 16:54:32,180 WARN [sync.1] wal.FSHLog$SyncRunner(589): UNEXPECTED 623 // java.lang.NullPointerException 624 // at org.apache.hadoop.hbase.regionserver.wal.FSHLog$SyncRunner.run(FSHLog.java:582) 625 // at java.lang.Thread.run(Thread.java:748) 626 SyncFuture sf; 627 while (true) { 628 takeSyncFuture = null; 629 // We have to process what we 'take' from the queue 630 takeSyncFuture = this.syncFutures.take(); 631 // Make local copy. 632 sf = takeSyncFuture; 633 currentSequence = this.sequence; 634 long syncFutureSequence = sf.getTxid(); 635 if (syncFutureSequence > currentSequence) { 636 throw new IllegalStateException("currentSequence=" + currentSequence 637 + ", syncFutureSequence=" + syncFutureSequence); 638 } 639 // See if we can process any syncfutures BEFORE we go sync. 640 long currentHighestSyncedSequence = highestSyncedTxid.get(); 641 if (currentSequence < currentHighestSyncedSequence) { 642 syncCount += releaseSyncFuture(sf, currentHighestSyncedSequence, null); 643 // Done with the 'take'. Go around again and do a new 'take'. 644 continue; 645 } 646 break; 647 } 648 // I got something. Lets run. Save off current sequence number in case it changes 649 // while we run. 650 long start = System.nanoTime(); 651 Throwable lastException = null; 652 try { 653 long unSyncedFlushSeq = highestUnsyncedTxid; 654 writer.sync(sf.isForceSync()); 655 if (unSyncedFlushSeq > currentSequence) { 656 currentSequence = unSyncedFlushSeq; 657 } 658 currentSequence = updateHighestSyncedSequence(currentSequence); 659 } catch (IOException e) { 660 LOG.error("Error syncing, request close of WAL", e); 661 lastException = e; 662 } catch (Exception e) { 663 LOG.warn("UNEXPECTED", e); 664 lastException = e; 665 } finally { 666 // First release what we 'took' from the queue. 667 syncCount += releaseSyncFuture(takeSyncFuture, currentSequence, lastException); 668 // Can we release other syncs? 669 syncCount += releaseSyncFutures(currentSequence, lastException); 670 if (lastException != null) { 671 requestLogRoll(ERROR); 672 } else { 673 checkLogRoll(); 674 } 675 } 676 postSync(System.nanoTime() - start, syncCount); 677 } catch (InterruptedException e) { 678 // Presume legit interrupt. 679 Thread.currentThread().interrupt(); 680 } catch (Throwable t) { 681 LOG.warn("UNEXPECTED, continuing", t); 682 } 683 } 684 } 685 } 686 687 /** 688 * Schedule a log roll if needed. 689 */ 690 private boolean checkLogRoll() { 691 // If we have already requested a roll, do nothing 692 if (isLogRollRequested()) { 693 return false; 694 } 695 // Will return immediately if we are in the middle of a WAL log roll currently. 696 if (!rollWriterLock.tryLock()) { 697 return false; 698 } 699 try { 700 if (doCheckLogLowReplication()) { 701 LOG.warn("Requesting log roll because of low replication, current pipeline: " 702 + Arrays.toString(getPipeline())); 703 requestLogRoll(LOW_REPLICATION); 704 return true; 705 } else if (writer != null && writer.getLength() > logrollsize) { 706 if (LOG.isDebugEnabled()) { 707 LOG.debug("Requesting log roll because of file size threshold; length=" 708 + writer.getLength() + ", logrollsize=" + logrollsize); 709 } 710 requestLogRoll(SIZE); 711 return true; 712 } else if (doCheckSlowSync()) { 713 // We log this already in checkSlowSync 714 requestLogRoll(SLOW_SYNC); 715 return true; 716 } 717 } finally { 718 rollWriterLock.unlock(); 719 } 720 return false; 721 } 722 723 /** Returns true if number of replicas for the WAL is lower than threshold */ 724 @Override 725 protected boolean doCheckLogLowReplication() { 726 boolean logRollNeeded = false; 727 // if the number of replicas in HDFS has fallen below the configured 728 // value, then roll logs. 729 try { 730 int numCurrentReplicas = getLogReplication(); 731 if (numCurrentReplicas != 0 && numCurrentReplicas < this.minTolerableReplication) { 732 if (this.lowReplicationRollEnabled) { 733 if (this.consecutiveLogRolls.get() < this.lowReplicationRollLimit) { 734 LOG.warn("HDFS pipeline error detected. " + "Found " + numCurrentReplicas 735 + " replicas but expecting no less than " + this.minTolerableReplication 736 + " replicas. " + " Requesting close of WAL. current pipeline: " 737 + Arrays.toString(getPipeline())); 738 logRollNeeded = true; 739 // If rollWriter is requested, increase consecutiveLogRolls. Once it 740 // is larger than lowReplicationRollLimit, disable the 741 // LowReplication-Roller 742 this.consecutiveLogRolls.getAndIncrement(); 743 } else { 744 LOG.warn("Too many consecutive RollWriter requests, it's a sign of " 745 + "the total number of live datanodes is lower than the tolerable replicas."); 746 this.consecutiveLogRolls.set(0); 747 this.lowReplicationRollEnabled = false; 748 } 749 } 750 } else if (numCurrentReplicas >= this.minTolerableReplication) { 751 if (!this.lowReplicationRollEnabled) { 752 // The new writer's log replicas is always the default value. 753 // So we should not enable LowReplication-Roller. If numEntries 754 // is lower than or equals 1, we consider it as a new writer. 755 if (this.numEntries.get() <= 1) { 756 return logRollNeeded; 757 } 758 // Once the live datanode number and the replicas return to normal, 759 // enable the LowReplication-Roller. 760 this.lowReplicationRollEnabled = true; 761 LOG.info("LowReplication-Roller was enabled."); 762 } 763 } 764 } catch (Exception e) { 765 LOG.warn("DFSOutputStream.getNumCurrentReplicas failed because of " + e + ", continuing..."); 766 } 767 return logRollNeeded; 768 } 769 770 protected long getSequenceOnRingBuffer() { 771 return this.disruptor.getRingBuffer().next(); 772 } 773 774 private SyncFuture publishSyncOnRingBuffer(boolean forceSync) { 775 long sequence = getSequenceOnRingBuffer(); 776 return publishSyncOnRingBuffer(sequence, forceSync); 777 } 778 779 protected SyncFuture publishSyncOnRingBuffer(long sequence, boolean forceSync) { 780 // here we use ring buffer sequence as transaction id 781 SyncFuture syncFuture = getSyncFuture(sequence, forceSync); 782 try { 783 RingBufferTruck truck = this.disruptor.getRingBuffer().get(sequence); 784 truck.load(syncFuture); 785 } finally { 786 this.disruptor.getRingBuffer().publish(sequence); 787 } 788 return syncFuture; 789 } 790 791 // Sync all known transactions 792 private void publishSyncThenBlockOnCompletion(boolean forceSync) throws IOException { 793 SyncFuture syncFuture = publishSyncOnRingBuffer(forceSync); 794 blockOnSync(syncFuture); 795 } 796 797 /** 798 * {@inheritDoc} 799 * <p> 800 * If the pipeline isn't started yet or is empty, you will get the default replication factor. 801 * Therefore, if this function returns 0, it means you are not properly running with the HDFS-826 802 * patch. 803 */ 804 @Override 805 int getLogReplication() { 806 try { 807 // in standalone mode, it will return 0 808 if (this.hdfs_out instanceof HdfsDataOutputStream) { 809 return ((HdfsDataOutputStream) this.hdfs_out).getCurrentBlockReplication(); 810 } 811 } catch (IOException e) { 812 LOG.info("", e); 813 } 814 return 0; 815 } 816 817 @Override 818 protected void doSync(boolean forceSync) throws IOException { 819 publishSyncThenBlockOnCompletion(forceSync); 820 } 821 822 @Override 823 protected void doSync(long txid, boolean forceSync) throws IOException { 824 if (this.highestSyncedTxid.get() >= txid) { 825 // Already sync'd. 826 return; 827 } 828 publishSyncThenBlockOnCompletion(forceSync); 829 } 830 831 boolean isLowReplicationRollEnabled() { 832 return lowReplicationRollEnabled; 833 } 834 835 public static final long FIXED_OVERHEAD = 836 ClassSize.align(ClassSize.OBJECT + (5 * ClassSize.REFERENCE) + (2 * ClassSize.ATOMIC_INTEGER) 837 + (3 * Bytes.SIZEOF_INT) + (4 * Bytes.SIZEOF_LONG)); 838 839 /** 840 * This class is used coordinating two threads holding one thread at a 'safe point' while the 841 * orchestrating thread does some work that requires the first thread paused: e.g. holding the WAL 842 * writer while its WAL is swapped out from under it by another thread. 843 * <p> 844 * Thread A signals Thread B to hold when it gets to a 'safe point'. Thread A wait until Thread B 845 * gets there. When the 'safe point' has been attained, Thread B signals Thread A. Thread B then 846 * holds at the 'safe point'. Thread A on notification that Thread B is paused, goes ahead and 847 * does the work it needs to do while Thread B is holding. When Thread A is done, it flags B and 848 * then Thread A and Thread B continue along on their merry way. Pause and signalling 'zigzags' 849 * between the two participating threads. We use two latches -- one the inverse of the other -- 850 * pausing and signaling when states are achieved. 851 * <p> 852 * To start up the drama, Thread A creates an instance of this class each time it would do this 853 * zigzag dance and passes it to Thread B (these classes use Latches so it is one shot only). 854 * Thread B notices the new instance (via reading a volatile reference or how ever) and it starts 855 * to work toward the 'safe point'. Thread A calls {@link #waitSafePoint(SyncFuture)} when it 856 * cannot proceed until the Thread B 'safe point' is attained. Thread A will be held inside in 857 * {@link #waitSafePoint(SyncFuture)} until Thread B reaches the 'safe point'. Once there, Thread 858 * B frees Thread A by calling {@link #safePointAttained()}. Thread A now knows Thread B is at the 859 * 'safe point' and that it is holding there (When Thread B calls {@link #safePointAttained()} it 860 * blocks here until Thread A calls {@link #releaseSafePoint()}). Thread A proceeds to do what it 861 * needs to do while Thread B is paused. When finished, it lets Thread B lose by calling 862 * {@link #releaseSafePoint()} and away go both Threads again. 863 */ 864 static class SafePointZigZagLatch { 865 /** 866 * Count down this latch when safe point attained. 867 */ 868 private volatile CountDownLatch safePointAttainedLatch = new CountDownLatch(1); 869 /** 870 * Latch to wait on. Will be released when we can proceed. 871 */ 872 private volatile CountDownLatch safePointReleasedLatch = new CountDownLatch(1); 873 874 private void checkIfSyncFailed(SyncFuture syncFuture) throws FailedSyncBeforeLogCloseException { 875 Throwable t = syncFuture.getThrowable(); 876 if (t != null) { 877 throw new FailedSyncBeforeLogCloseException(t); 878 } 879 } 880 881 /** 882 * For Thread A to call when it is ready to wait on the 'safe point' to be attained. Thread A 883 * will be held in here until Thread B calls {@link #safePointAttained()} 884 * @param syncFuture We need this as barometer on outstanding syncs. If it comes home with an 885 * exception, then something is up w/ our syncing. 886 * @return The passed <code>syncFuture</code> 887 */ 888 SyncFuture waitSafePoint(SyncFuture syncFuture) 889 throws InterruptedException, FailedSyncBeforeLogCloseException { 890 while (!this.safePointAttainedLatch.await(1, TimeUnit.MILLISECONDS)) { 891 checkIfSyncFailed(syncFuture); 892 } 893 checkIfSyncFailed(syncFuture); 894 return syncFuture; 895 } 896 897 /** Returns if the safepoint has been attained. */ 898 @InterfaceAudience.Private 899 boolean isSafePointAttained() { 900 return this.safePointAttainedLatch.getCount() == 0; 901 } 902 903 /** 904 * Called by Thread B when it attains the 'safe point'. In this method, Thread B signals Thread 905 * A it can proceed. Thread B will be held in here until {@link #releaseSafePoint()} is called 906 * by Thread A. 907 */ 908 void safePointAttained() throws InterruptedException { 909 this.safePointAttainedLatch.countDown(); 910 this.safePointReleasedLatch.await(); 911 } 912 913 /** 914 * Called by Thread A when it is done with the work it needs to do while Thread B is halted. 915 * This will release the Thread B held in a call to {@link #safePointAttained()} 916 */ 917 void releaseSafePoint() { 918 this.safePointReleasedLatch.countDown(); 919 } 920 921 /** Returns True is this is a 'cocked', fresh instance, and not one that has already fired. */ 922 boolean isCocked() { 923 return this.safePointAttainedLatch.getCount() > 0 924 && this.safePointReleasedLatch.getCount() > 0; 925 } 926 } 927 928 /** 929 * Handler that is run by the disruptor ringbuffer consumer. Consumer is a SINGLE 930 * 'writer/appender' thread. Appends edits and starts up sync runs. Tries its best to batch up 931 * syncs. There is no discernible benefit batching appends so we just append as they come in 932 * because it simplifies the below implementation. See metrics for batching effectiveness (In 933 * measurement, at 100 concurrent handlers writing 1k, we are batching > 10 appends and 10 handler 934 * sync invocations for every actual dfsclient sync call; at 10 concurrent handlers, YMMV). 935 * <p> 936 * Herein, we have an array into which we store the sync futures as they come in. When we have a 937 * 'batch', we'll then pass what we have collected to a SyncRunner thread to do the filesystem 938 * sync. When it completes, it will then call {@link SyncFuture#done(long, Throwable)} on each of 939 * SyncFutures in the batch to release blocked Handler threads. 940 * <p> 941 * I've tried various effects to try and make latencies low while keeping throughput high. I've 942 * tried keeping a single Queue of SyncFutures in this class appending to its tail as the syncs 943 * coming and having sync runner threads poll off the head to 'finish' completed SyncFutures. I've 944 * tried linkedlist, and various from concurrent utils whether LinkedBlockingQueue or 945 * ArrayBlockingQueue, etc. The more points of synchronization, the more 'work' (according to 946 * 'perf stats') that has to be done; small increases in stall percentages seem to have a big 947 * impact on throughput/latencies. The below model where we have an array into which we stash the 948 * syncs and then hand them off to the sync thread seemed like a decent compromise. See HBASE-8755 949 * for more detail. 950 */ 951 class RingBufferEventHandler implements EventHandler<RingBufferTruck>, LifecycleAware { 952 private final SyncRunner[] syncRunners; 953 private final SyncFuture[] syncFutures; 954 // Had 'interesting' issues when this was non-volatile. On occasion, we'd not pass all 955 // syncFutures to the next sync'ing thread. 956 private AtomicInteger syncFuturesCount = new AtomicInteger(); 957 private volatile SafePointZigZagLatch zigzagLatch; 958 /** 959 * Set if we get an exception appending or syncing so that all subsequence appends and syncs on 960 * this WAL fail until WAL is replaced. 961 */ 962 private Exception exception = null; 963 /** 964 * Object to block on while waiting on safe point. 965 */ 966 private final Object safePointWaiter = new Object(); 967 private volatile boolean shutdown = false; 968 969 /** 970 * Which syncrunner to use next. 971 */ 972 private int syncRunnerIndex; 973 974 RingBufferEventHandler(final int syncRunnerCount, final int maxBatchCount) { 975 this.syncFutures = new SyncFuture[maxBatchCount]; 976 this.syncRunners = new SyncRunner[syncRunnerCount]; 977 for (int i = 0; i < syncRunnerCount; i++) { 978 this.syncRunners[i] = new SyncRunner("sync." + i, maxBatchCount); 979 } 980 } 981 982 private void cleanupOutstandingSyncsOnException(final long sequence, final Exception e) { 983 // There could be handler-count syncFutures outstanding. 984 for (int i = 0; i < this.syncFuturesCount.get(); i++) { 985 this.syncFutures[i].done(sequence, e); 986 } 987 offerDoneSyncsBackToCache(); 988 } 989 990 /** 991 * Offers the finished syncs back to the cache for reuse. 992 */ 993 private void offerDoneSyncsBackToCache() { 994 for (int i = 0; i < this.syncFuturesCount.get(); i++) { 995 syncFutureCache.offer(syncFutures[i]); 996 } 997 this.syncFuturesCount.set(0); 998 } 999 1000 /** Returns True if outstanding sync futures still */ 1001 private boolean isOutstandingSyncs() { 1002 // Look at SyncFutures in the EventHandler 1003 for (int i = 0; i < this.syncFuturesCount.get(); i++) { 1004 if (!this.syncFutures[i].isDone()) { 1005 return true; 1006 } 1007 } 1008 1009 return false; 1010 } 1011 1012 private boolean isOutstandingSyncsFromRunners() { 1013 // Look at SyncFutures in the SyncRunners 1014 for (SyncRunner syncRunner : syncRunners) { 1015 if (syncRunner.isAlive() && !syncRunner.areSyncFuturesReleased()) { 1016 return true; 1017 } 1018 } 1019 return false; 1020 } 1021 1022 @Override 1023 // We can set endOfBatch in the below method if at end of our this.syncFutures array 1024 public void onEvent(final RingBufferTruck truck, final long sequence, boolean endOfBatch) 1025 throws Exception { 1026 // Appends and syncs are coming in order off the ringbuffer. We depend on this fact. We'll 1027 // add appends to dfsclient as they come in. Batching appends doesn't give any significant 1028 // benefit on measurement. Handler sync calls we will batch up. If we get an exception 1029 // appending an edit, we fail all subsequent appends and syncs with the same exception until 1030 // the WAL is reset. It is important that we not short-circuit and exit early this method. 1031 // It is important that we always go through the attainSafePoint on the end. Another thread, 1032 // the log roller may be waiting on a signal from us here and will just hang without it. 1033 1034 try { 1035 if (truck.type() == RingBufferTruck.Type.SYNC) { 1036 this.syncFutures[this.syncFuturesCount.getAndIncrement()] = truck.unloadSync(); 1037 // Force flush of syncs if we are carrying a full complement of syncFutures. 1038 if (this.syncFuturesCount.get() == this.syncFutures.length) { 1039 endOfBatch = true; 1040 } 1041 } else if (truck.type() == RingBufferTruck.Type.APPEND) { 1042 FSWALEntry entry = truck.unloadAppend(); 1043 try { 1044 if (this.exception != null) { 1045 // Return to keep processing events coming off the ringbuffer 1046 return; 1047 } 1048 append(entry); 1049 } catch (Exception e) { 1050 // Failed append. Record the exception. 1051 this.exception = e; 1052 // invoking cleanupOutstandingSyncsOnException when append failed with exception, 1053 // it will cleanup existing sync requests recorded in syncFutures but not offered to 1054 // SyncRunner yet, 1055 // so there won't be any sync future left over if no further truck published to 1056 // disruptor. 1057 cleanupOutstandingSyncsOnException(sequence, 1058 this.exception instanceof DamagedWALException 1059 ? this.exception 1060 : new DamagedWALException("On sync", this.exception)); 1061 // Return to keep processing events coming off the ringbuffer 1062 return; 1063 } finally { 1064 entry.release(); 1065 } 1066 } else { 1067 // What is this if not an append or sync. Fail all up to this!!! 1068 cleanupOutstandingSyncsOnException(sequence, 1069 new IllegalStateException("Neither append nor sync")); 1070 // Return to keep processing. 1071 return; 1072 } 1073 1074 // TODO: Check size and if big go ahead and call a sync if we have enough data. 1075 // This is a sync. If existing exception, fall through. Else look to see if batch. 1076 if (this.exception == null) { 1077 // If not a batch, return to consume more events from the ring buffer before proceeding; 1078 // we want to get up a batch of syncs and appends before we go do a filesystem sync. 1079 if (!endOfBatch || this.syncFuturesCount.get() <= 0) { 1080 return; 1081 } 1082 // syncRunnerIndex is bound to the range [0, Integer.MAX_INT - 1] as follows: 1083 // * The maximum value possible for syncRunners.length is Integer.MAX_INT 1084 // * syncRunnerIndex starts at 0 and is incremented only here 1085 // * after the increment, the value is bounded by the '%' operator to 1086 // [0, syncRunners.length), presuming the value was positive prior to 1087 // the '%' operator. 1088 // * after being bound to [0, Integer.MAX_INT - 1], the new value is stored in 1089 // syncRunnerIndex ensuring that it can't grow without bound and overflow. 1090 // * note that the value after the increment must be positive, because the most it 1091 // could have been prior was Integer.MAX_INT - 1 and we only increment by 1. 1092 this.syncRunnerIndex = (this.syncRunnerIndex + 1) % this.syncRunners.length; 1093 try { 1094 // Below expects that the offer 'transfers' responsibility for the outstanding syncs to 1095 // the syncRunner. We should never get an exception in here. 1096 this.syncRunners[this.syncRunnerIndex].offer(sequence, this.syncFutures, 1097 this.syncFuturesCount.get()); 1098 } catch (Exception e) { 1099 // Should NEVER get here. 1100 requestLogRoll(ERROR); 1101 this.exception = new DamagedWALException("Failed offering sync", e); 1102 } 1103 } 1104 // We may have picked up an exception above trying to offer sync 1105 if (this.exception != null) { 1106 cleanupOutstandingSyncsOnException(sequence, 1107 this.exception instanceof DamagedWALException 1108 ? this.exception 1109 : new DamagedWALException("On sync", this.exception)); 1110 } 1111 attainSafePoint(sequence); 1112 // It is critical that we offer the futures back to the cache for reuse here after the 1113 // safe point is attained and all the clean up has been done. There have been 1114 // issues with reusing sync futures early causing WAL lockups, see HBASE-25984. 1115 offerDoneSyncsBackToCache(); 1116 } catch (Throwable t) { 1117 LOG.error("UNEXPECTED!!! syncFutures.length=" + this.syncFutures.length, t); 1118 } 1119 } 1120 1121 SafePointZigZagLatch attainSafePoint() { 1122 this.zigzagLatch = new SafePointZigZagLatch(); 1123 return this.zigzagLatch; 1124 } 1125 1126 /** 1127 * Check if we should attain safe point. If so, go there and then wait till signalled before we 1128 * proceeding. 1129 */ 1130 private void attainSafePoint(final long currentSequence) { 1131 if (this.zigzagLatch == null || !this.zigzagLatch.isCocked()) { 1132 return; 1133 } 1134 // If here, another thread is waiting on us to get to safe point. Don't leave it hanging. 1135 beforeWaitOnSafePoint(); 1136 try { 1137 // Wait on outstanding syncers; wait for them to finish syncing (unless we've been 1138 // shutdown or unless our latch has been thrown because we have been aborted or unless 1139 // this WAL is broken and we can't get a sync/append to complete). 1140 while ( 1141 (!this.shutdown && this.zigzagLatch.isCocked() 1142 && highestSyncedTxid.get() < currentSequence && 1143 // We could be in here and all syncs are failing or failed. Check for this. Otherwise 1144 // we'll just be stuck here for ever. In other words, ensure there syncs running. 1145 isOutstandingSyncs()) 1146 // Wait for all SyncRunners to finish their work so that we can replace the writer 1147 || isOutstandingSyncsFromRunners() 1148 ) { 1149 synchronized (this.safePointWaiter) { 1150 this.safePointWaiter.wait(0, 1); 1151 } 1152 } 1153 // Tell waiting thread we've attained safe point. Can clear this.throwable if set here 1154 // because we know that next event through the ringbuffer will be going to a new WAL 1155 // after we do the zigzaglatch dance. 1156 this.exception = null; 1157 this.zigzagLatch.safePointAttained(); 1158 } catch (InterruptedException e) { 1159 LOG.warn("Interrupted ", e); 1160 Thread.currentThread().interrupt(); 1161 } 1162 } 1163 1164 /** 1165 * Append to the WAL. Does all CP and WAL listener calls. 1166 */ 1167 void append(final FSWALEntry entry) throws Exception { 1168 try { 1169 FSHLog.this.appendEntry(writer, entry); 1170 } catch (Exception e) { 1171 String msg = 1172 "Append sequenceId=" + entry.getKey().getSequenceId() + ", requesting roll of WAL"; 1173 LOG.warn(msg, e); 1174 requestLogRoll(ERROR); 1175 throw new DamagedWALException(msg, e); 1176 } 1177 } 1178 1179 @Override 1180 public void onStart() { 1181 for (SyncRunner syncRunner : this.syncRunners) { 1182 syncRunner.start(); 1183 } 1184 } 1185 1186 @Override 1187 public void onShutdown() { 1188 for (SyncRunner syncRunner : this.syncRunners) { 1189 syncRunner.interrupt(); 1190 } 1191 } 1192 } 1193 1194 /** 1195 * This method gets the pipeline for the current WAL. 1196 */ 1197 @Override 1198 DatanodeInfo[] getPipeline() { 1199 if (this.hdfs_out != null) { 1200 if (this.hdfs_out.getWrappedStream() instanceof DFSOutputStream) { 1201 return ((DFSOutputStream) this.hdfs_out.getWrappedStream()).getPipeline(); 1202 } 1203 } 1204 return new DatanodeInfo[0]; 1205 } 1206 1207 Writer getWriter() { 1208 return this.writer; 1209 } 1210 1211 void setWriter(Writer writer) { 1212 this.writer = writer; 1213 } 1214}