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