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