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