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