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