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