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}