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