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