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