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.RingBuffer;
021import com.lmax.disruptor.Sequence;
022import com.lmax.disruptor.Sequencer;
023import java.io.IOException;
024import java.lang.reflect.Field;
025import java.util.ArrayDeque;
026import java.util.Comparator;
027import java.util.Deque;
028import java.util.Iterator;
029import java.util.List;
030import java.util.Queue;
031import java.util.SortedSet;
032import java.util.TreeSet;
033import java.util.concurrent.ExecutorService;
034import java.util.concurrent.Executors;
035import java.util.concurrent.LinkedBlockingQueue;
036import java.util.concurrent.ThreadPoolExecutor;
037import java.util.concurrent.TimeUnit;
038import java.util.concurrent.atomic.AtomicBoolean;
039import java.util.concurrent.locks.Condition;
040import java.util.concurrent.locks.Lock;
041import java.util.concurrent.locks.ReentrantLock;
042import java.util.function.Supplier;
043import org.apache.hadoop.conf.Configuration;
044import org.apache.hadoop.fs.FileSystem;
045import org.apache.hadoop.fs.Path;
046import org.apache.hadoop.hbase.HBaseInterfaceAudience;
047import org.apache.hadoop.hbase.client.RegionInfo;
048import org.apache.hadoop.hbase.io.asyncfs.AsyncFSOutput;
049import org.apache.hadoop.hbase.trace.TraceUtil;
050import org.apache.hadoop.hbase.wal.AsyncFSWALProvider;
051import org.apache.hadoop.hbase.wal.WALEdit;
052import org.apache.hadoop.hbase.wal.WALKeyImpl;
053import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter;
054import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
055import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
056import org.apache.htrace.core.TraceScope;
057import org.apache.yetus.audience.InterfaceAudience;
058import org.slf4j.Logger;
059import org.slf4j.LoggerFactory;
060
061import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
062import org.apache.hbase.thirdparty.io.netty.channel.Channel;
063import org.apache.hbase.thirdparty.io.netty.channel.EventLoop;
064import org.apache.hbase.thirdparty.io.netty.channel.EventLoopGroup;
065import org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor;
066
067/**
068 * An asynchronous implementation of FSWAL.
069 * <p>
070 * Here 'waitingConsumePayloads' acts as the RingBuffer in FSHLog.
071 * <p>
072 * For append, we process it as follow:
073 * <ol>
074 * <li>In the caller thread(typically, in the rpc handler thread):
075 * <ol>
076 * <li>Insert the entry into 'waitingConsumePayloads'. Use ringbuffer sequence as txid.</li>
077 * <li>Schedule the consumer task if needed. See {@link #shouldScheduleConsumer()} for more details.
078 * </li>
079 * </ol>
080 * </li>
081 * <li>In the consumer task(executed in a single threaded thread pool)
082 * <ol>
083 * <li>Poll the entry from {@link #waitingConsumePayloads} and insert it into
084 * {@link #toWriteAppends}</li>
085 * <li>Poll the entry from {@link #toWriteAppends}, append it to the AsyncWriter, and insert it into
086 * {@link #unackedAppends}</li>
087 * <li>If the buffered size reaches {@link #batchSize}, or there is a sync request, then we call
088 * sync on the AsyncWriter.</li>
089 * <li>In the callback methods:
090 * <ul>
091 * <li>If succeeded, poll the entry from {@link #unackedAppends} and drop it.</li>
092 * <li>If failed, add all the entries in {@link #unackedAppends} back to {@link #toWriteAppends} and
093 * wait for writing them again.</li>
094 * </ul>
095 * </li>
096 * </ol>
097 * </li>
098 * </ol>
099 * For sync, the processing stages are almost same. And different from FSHLog, we will open a new
100 * writer and rewrite unacked entries to the new writer and sync again if we hit a sync error.
101 * <p>
102 * Here we only describe the logic of doReplaceWriter. The main logic of rollWriter is same with
103 * FSHLog.<br>
104 * For a normal roll request(for example, we have reached the log roll size):
105 * <ol>
106 * <li>In the log roller thread, we will set {@link #waitingRoll} to true and
107 * {@link #readyForRolling} to false, and then wait on {@link #readyForRolling}(see
108 * {@link #waitForSafePoint()}).</li>
109 * <li>In the consumer thread, we will stop polling entries from {@link #waitingConsumePayloads} if
110 * {@link #waitingRoll} is true, and also stop writing the entries in {@link #toWriteAppends} out.
111 * </li>
112 * <li>If there are unflush data in the writer, sync them.</li>
113 * <li>When all out-going sync request is finished, i.e, the {@link #unackedAppends} is empty,
114 * signal the {@link #readyForRollingCond}.</li>
115 * <li>Back to the log roller thread, now we can confirm that there are no out-going entries, i.e.,
116 * we reach a safe point. So it is safe to replace old writer with new writer now.</li>
117 * <li>Set {@link #writerBroken} and {@link #waitingRoll} to false.</li>
118 * <li>Schedule the consumer task.</li>
119 * <li>Schedule a background task to close the old writer.</li>
120 * </ol>
121 * For a broken writer roll request, the only difference is that we can bypass the wait for safe
122 * point stage.
123 */
124@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
125public class AsyncFSWAL extends AbstractFSWAL<AsyncWriter> {
126
127  private static final Logger LOG = LoggerFactory.getLogger(AsyncFSWAL.class);
128
129  private static final Comparator<SyncFuture> SEQ_COMPARATOR = (o1, o2) -> {
130    int c = Long.compare(o1.getTxid(), o2.getTxid());
131    return c != 0 ? c : Integer.compare(System.identityHashCode(o1), System.identityHashCode(o2));
132  };
133
134  public static final String WAL_BATCH_SIZE = "hbase.wal.batch.size";
135  public static final long DEFAULT_WAL_BATCH_SIZE = 64L * 1024;
136
137  public static final String ASYNC_WAL_USE_SHARED_EVENT_LOOP =
138    "hbase.wal.async.use-shared-event-loop";
139  public static final boolean DEFAULT_ASYNC_WAL_USE_SHARED_EVENT_LOOP = false;
140
141  public static final String ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS =
142    "hbase.wal.async.wait.on.shutdown.seconds";
143  public static final int DEFAULT_ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS = 5;
144
145  private final EventLoopGroup eventLoopGroup;
146
147  private final ExecutorService consumeExecutor;
148
149  private final Class<? extends Channel> channelClass;
150
151  private final Lock consumeLock = new ReentrantLock();
152
153  private final Runnable consumer = this::consume;
154
155  // check if there is already a consumer task in the event loop's task queue
156  private final Supplier<Boolean> hasConsumerTask;
157
158  private static final int MAX_EPOCH = 0x3FFFFFFF;
159  // the lowest bit is waitingRoll, which means new writer is created and we are waiting for old
160  // writer to be closed.
161  // the second lowest bit is writerBorken which means the current writer is broken and rollWriter
162  // is needed.
163  // all other bits are the epoch number of the current writer, this is used to detect whether the
164  // writer is still the one when you issue the sync.
165  // notice that, modification to this field is only allowed under the protection of consumeLock.
166  private volatile int epochAndState;
167
168  // used to guard the log roll request when we exceed the log roll size.
169  private boolean rollRequested;
170
171  private boolean readyForRolling;
172
173  private final Condition readyForRollingCond = consumeLock.newCondition();
174
175  private final RingBuffer<RingBufferTruck> waitingConsumePayloads;
176
177  private final Sequence waitingConsumePayloadsGatingSequence;
178
179  private final AtomicBoolean consumerScheduled = new AtomicBoolean(false);
180
181  private final long batchSize;
182
183  private final ExecutorService closeExecutor = Executors.newCachedThreadPool(
184    new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Close-WAL-Writer-%d").build());
185
186  private volatile AsyncFSOutput fsOut;
187
188  private final Deque<FSWALEntry> toWriteAppends = new ArrayDeque<>();
189
190  private final Deque<FSWALEntry> unackedAppends = new ArrayDeque<>();
191
192  private final SortedSet<SyncFuture> syncFutures = new TreeSet<>(SEQ_COMPARATOR);
193
194  // the highest txid of WAL entries being processed
195  private long highestProcessedAppendTxid;
196
197  // file length when we issue last sync request on the writer
198  private long fileLengthAtLastSync;
199
200  private long highestProcessedAppendTxidAtLastSync;
201
202  private final int waitOnShutdownInSeconds;
203
204  public AsyncFSWAL(FileSystem fs, Path rootDir, String logDir, String archiveDir,
205      Configuration conf, List<WALActionsListener> listeners, boolean failIfWALExists,
206      String prefix, String suffix, EventLoopGroup eventLoopGroup,
207      Class<? extends Channel> channelClass) throws FailedLogCloseException, IOException {
208    super(fs, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix);
209    this.eventLoopGroup = eventLoopGroup;
210    this.channelClass = channelClass;
211    Supplier<Boolean> hasConsumerTask;
212    if (conf.getBoolean(ASYNC_WAL_USE_SHARED_EVENT_LOOP, DEFAULT_ASYNC_WAL_USE_SHARED_EVENT_LOOP)) {
213      this.consumeExecutor = eventLoopGroup.next();
214      if (consumeExecutor instanceof SingleThreadEventExecutor) {
215        try {
216          Field field = SingleThreadEventExecutor.class.getDeclaredField("taskQueue");
217          field.setAccessible(true);
218          Queue<?> queue = (Queue<?>) field.get(consumeExecutor);
219          hasConsumerTask = () -> queue.peek() == consumer;
220        } catch (Exception e) {
221          LOG.warn("Can not get task queue of " + consumeExecutor +
222            ", this is not necessary, just give up", e);
223          hasConsumerTask = () -> false;
224        }
225      } else {
226        hasConsumerTask = () -> false;
227      }
228    } else {
229      ThreadPoolExecutor threadPool =
230        new ThreadPoolExecutor(1, 1, 0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>(),
231            new ThreadFactoryBuilder().setNameFormat("AsyncFSWAL-%d").setDaemon(true).build());
232      hasConsumerTask = () -> threadPool.getQueue().peek() == consumer;
233      this.consumeExecutor = threadPool;
234    }
235
236    this.hasConsumerTask = hasConsumerTask;
237    int preallocatedEventCount =
238      conf.getInt("hbase.regionserver.wal.disruptor.event.count", 1024 * 16);
239    waitingConsumePayloads =
240      RingBuffer.createMultiProducer(RingBufferTruck::new, preallocatedEventCount);
241    waitingConsumePayloadsGatingSequence = new Sequence(Sequencer.INITIAL_CURSOR_VALUE);
242    waitingConsumePayloads.addGatingSequences(waitingConsumePayloadsGatingSequence);
243
244    // inrease the ringbuffer sequence so our txid is start from 1
245    waitingConsumePayloads.publish(waitingConsumePayloads.next());
246    waitingConsumePayloadsGatingSequence.set(waitingConsumePayloads.getCursor());
247
248    batchSize = conf.getLong(WAL_BATCH_SIZE, DEFAULT_WAL_BATCH_SIZE);
249    waitOnShutdownInSeconds = conf.getInt(ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS,
250      DEFAULT_ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS);
251    rollWriter();
252  }
253
254  private static boolean waitingRoll(int epochAndState) {
255    return (epochAndState & 1) != 0;
256  }
257
258  private static boolean writerBroken(int epochAndState) {
259    return ((epochAndState >>> 1) & 1) != 0;
260  }
261
262  private static int epoch(int epochAndState) {
263    return epochAndState >>> 2;
264  }
265
266  // return whether we have successfully set readyForRolling to true.
267  private boolean trySetReadyForRolling() {
268    // Check without holding lock first. Usually we will just return here.
269    // waitingRoll is volatile and unacedEntries is only accessed inside event loop so it is safe to
270    // check them outside the consumeLock.
271    if (!waitingRoll(epochAndState) || !unackedAppends.isEmpty()) {
272      return false;
273    }
274    consumeLock.lock();
275    try {
276      // 1. a roll is requested
277      // 2. all out-going entries have been acked(we have confirmed above).
278      if (waitingRoll(epochAndState)) {
279        readyForRolling = true;
280        readyForRollingCond.signalAll();
281        return true;
282      } else {
283        return false;
284      }
285    } finally {
286      consumeLock.unlock();
287    }
288  }
289
290  private void syncFailed(long epochWhenSync, Throwable error) {
291    LOG.warn("sync failed", error);
292    boolean shouldRequestLogRoll = true;
293    consumeLock.lock();
294    try {
295      int currentEpochAndState = epochAndState;
296      if (epoch(currentEpochAndState) != epochWhenSync || writerBroken(currentEpochAndState)) {
297        // this is not the previous writer which means we have already rolled the writer.
298        // or this is still the current writer, but we have already marked it as broken and request
299        // a roll.
300        return;
301      }
302      this.epochAndState = currentEpochAndState | 0b10;
303      if (waitingRoll(currentEpochAndState)) {
304        readyForRolling = true;
305        readyForRollingCond.signalAll();
306        // this means we have already in the middle of a rollWriter so just tell the roller thread
307        // that you can continue without requesting an extra log roll.
308        shouldRequestLogRoll = false;
309      }
310    } finally {
311      consumeLock.unlock();
312    }
313    for (Iterator<FSWALEntry> iter = unackedAppends.descendingIterator(); iter.hasNext();) {
314      toWriteAppends.addFirst(iter.next());
315    }
316    highestUnsyncedTxid = highestSyncedTxid.get();
317    if (shouldRequestLogRoll) {
318      // request a roll.
319      requestLogRoll();
320    }
321  }
322
323  private void syncCompleted(AsyncWriter writer, long processedTxid, long startTimeNs) {
324    highestSyncedTxid.set(processedTxid);
325    for (Iterator<FSWALEntry> iter = unackedAppends.iterator(); iter.hasNext();) {
326      if (iter.next().getTxid() <= processedTxid) {
327        iter.remove();
328      } else {
329        break;
330      }
331    }
332    postSync(System.nanoTime() - startTimeNs, finishSync(true));
333    if (trySetReadyForRolling()) {
334      // we have just finished a roll, then do not need to check for log rolling, the writer will be
335      // closed soon.
336      return;
337    }
338    if (writer.getLength() < logrollsize || rollRequested) {
339      return;
340    }
341    rollRequested = true;
342    requestLogRoll();
343  }
344
345  private void sync(AsyncWriter writer) {
346    fileLengthAtLastSync = writer.getLength();
347    long currentHighestProcessedAppendTxid = highestProcessedAppendTxid;
348    highestProcessedAppendTxidAtLastSync = currentHighestProcessedAppendTxid;
349    final long startTimeNs = System.nanoTime();
350    final long epoch = (long) epochAndState >>> 2L;
351    writer.sync().whenCompleteAsync((result, error) -> {
352      if (error != null) {
353        syncFailed(epoch, error);
354      } else {
355        syncCompleted(writer, currentHighestProcessedAppendTxid, startTimeNs);
356      }
357    }, consumeExecutor);
358  }
359
360  private void addTimeAnnotation(SyncFuture future, String annotation) {
361    TraceUtil.addTimelineAnnotation(annotation);
362    // TODO handle htrace API change, see HBASE-18895
363    // future.setSpan(scope.getSpan());
364  }
365
366  private int finishSyncLowerThanTxid(long txid, boolean addSyncTrace) {
367    int finished = 0;
368    for (Iterator<SyncFuture> iter = syncFutures.iterator(); iter.hasNext();) {
369      SyncFuture sync = iter.next();
370      if (sync.getTxid() <= txid) {
371        sync.done(txid, null);
372        iter.remove();
373        finished++;
374        if (addSyncTrace) {
375          addTimeAnnotation(sync, "writer synced");
376        }
377      } else {
378        break;
379      }
380    }
381    return finished;
382  }
383
384  // try advancing the highestSyncedTxid as much as possible
385  private int finishSync(boolean addSyncTrace) {
386    if (unackedAppends.isEmpty()) {
387      // All outstanding appends have been acked.
388      if (toWriteAppends.isEmpty()) {
389        // Also no appends that wait to be written out, then just finished all pending syncs.
390        long maxSyncTxid = highestSyncedTxid.get();
391        for (SyncFuture sync : syncFutures) {
392          maxSyncTxid = Math.max(maxSyncTxid, sync.getTxid());
393          sync.done(maxSyncTxid, null);
394          if (addSyncTrace) {
395            addTimeAnnotation(sync, "writer synced");
396          }
397        }
398        highestSyncedTxid.set(maxSyncTxid);
399        int finished = syncFutures.size();
400        syncFutures.clear();
401        return finished;
402      } else {
403        // There is no append between highestProcessedAppendTxid and lowestUnprocessedAppendTxid, so
404        // if highestSyncedTxid >= highestProcessedAppendTxid, then all syncs whose txid are between
405        // highestProcessedAppendTxid and lowestUnprocessedAppendTxid can be finished.
406        long lowestUnprocessedAppendTxid = toWriteAppends.peek().getTxid();
407        assert lowestUnprocessedAppendTxid > highestProcessedAppendTxid;
408        long doneTxid = lowestUnprocessedAppendTxid - 1;
409        highestSyncedTxid.set(doneTxid);
410        return finishSyncLowerThanTxid(doneTxid, addSyncTrace);
411      }
412    } else {
413      // There are still unacked appends. So let's move the highestSyncedTxid to the txid of the
414      // first unacked append minus 1.
415      long lowestUnackedAppendTxid = unackedAppends.peek().getTxid();
416      long doneTxid = Math.max(lowestUnackedAppendTxid - 1, highestSyncedTxid.get());
417      highestSyncedTxid.set(doneTxid);
418      return finishSyncLowerThanTxid(doneTxid, addSyncTrace);
419    }
420  }
421
422  private void appendAndSync() {
423    final AsyncWriter writer = this.writer;
424    // maybe a sync request is not queued when we issue a sync, so check here to see if we could
425    // finish some.
426    finishSync(false);
427    long newHighestProcessedAppendTxid = -1L;
428    for (Iterator<FSWALEntry> iter = toWriteAppends.iterator(); iter.hasNext();) {
429      FSWALEntry entry = iter.next();
430      boolean appended;
431      try {
432        appended = append(writer, entry);
433      } catch (IOException e) {
434        throw new AssertionError("should not happen", e);
435      }
436      newHighestProcessedAppendTxid = entry.getTxid();
437      iter.remove();
438      if (appended) {
439        unackedAppends.addLast(entry);
440        if (writer.getLength() - fileLengthAtLastSync >= batchSize) {
441          break;
442        }
443      }
444    }
445    // if we have a newer transaction id, update it.
446    // otherwise, use the previous transaction id.
447    if (newHighestProcessedAppendTxid > 0) {
448      highestProcessedAppendTxid = newHighestProcessedAppendTxid;
449    } else {
450      newHighestProcessedAppendTxid = highestProcessedAppendTxid;
451    }
452
453    if (writer.getLength() - fileLengthAtLastSync >= batchSize) {
454      // sync because buffer size limit.
455      sync(writer);
456      return;
457    }
458    if (writer.getLength() == fileLengthAtLastSync) {
459      // we haven't written anything out, just advance the highestSyncedSequence since we may only
460      // stamped some region sequence id.
461      if (unackedAppends.isEmpty()) {
462        highestSyncedTxid.set(highestProcessedAppendTxid);
463        finishSync(false);
464        trySetReadyForRolling();
465      }
466      return;
467    }
468    // reach here means that we have some unsynced data but haven't reached the batch size yet
469    // but we will not issue a sync directly here even if there are sync requests because we may
470    // have some new data in the ringbuffer, so let's just return here and delay the decision of
471    // whether to issue a sync in the caller method.
472  }
473
474  private void consume() {
475    consumeLock.lock();
476    try {
477      int currentEpochAndState = epochAndState;
478      if (writerBroken(currentEpochAndState)) {
479        return;
480      }
481      if (waitingRoll(currentEpochAndState)) {
482        if (writer.getLength() > fileLengthAtLastSync) {
483          // issue a sync
484          sync(writer);
485        } else {
486          if (unackedAppends.isEmpty()) {
487            readyForRolling = true;
488            readyForRollingCond.signalAll();
489          }
490        }
491        return;
492      }
493    } finally {
494      consumeLock.unlock();
495    }
496    long nextCursor = waitingConsumePayloadsGatingSequence.get() + 1;
497    for (long cursorBound = waitingConsumePayloads.getCursor(); nextCursor <= cursorBound;
498      nextCursor++) {
499      if (!waitingConsumePayloads.isPublished(nextCursor)) {
500        break;
501      }
502      RingBufferTruck truck = waitingConsumePayloads.get(nextCursor);
503      switch (truck.type()) {
504        case APPEND:
505          toWriteAppends.addLast(truck.unloadAppend());
506          break;
507        case SYNC:
508          syncFutures.add(truck.unloadSync());
509          break;
510        default:
511          LOG.warn("RingBufferTruck with unexpected type: " + truck.type());
512          break;
513      }
514      waitingConsumePayloadsGatingSequence.set(nextCursor);
515    }
516    appendAndSync();
517    if (hasConsumerTask.get()) {
518      return;
519    }
520    if (toWriteAppends.isEmpty()) {
521      if (waitingConsumePayloadsGatingSequence.get() == waitingConsumePayloads.getCursor()) {
522        consumerScheduled.set(false);
523        // recheck here since in append and sync we do not hold the consumeLock. Thing may
524        // happen like
525        // 1. we check cursor, no new entry
526        // 2. someone publishes a new entry to ringbuffer and the consumerScheduled is true and
527        // give up scheduling the consumer task.
528        // 3. we set consumerScheduled to false and also give up scheduling consumer task.
529        if (waitingConsumePayloadsGatingSequence.get() == waitingConsumePayloads.getCursor()) {
530          // we will give up consuming so if there are some unsynced data we need to issue a sync.
531          if (writer.getLength() > fileLengthAtLastSync && !syncFutures.isEmpty() &&
532            syncFutures.last().getTxid() > highestProcessedAppendTxidAtLastSync) {
533            // no new data in the ringbuffer and we have at least one sync request
534            sync(writer);
535          }
536          return;
537        } else {
538          // maybe someone has grabbed this before us
539          if (!consumerScheduled.compareAndSet(false, true)) {
540            return;
541          }
542        }
543      }
544    }
545    // reschedule if we still have something to write.
546    consumeExecutor.execute(consumer);
547  }
548
549  private boolean shouldScheduleConsumer() {
550    int currentEpochAndState = epochAndState;
551    if (writerBroken(currentEpochAndState) || waitingRoll(currentEpochAndState)) {
552      return false;
553    }
554    return consumerScheduled.compareAndSet(false, true);
555  }
556
557  @Override
558  public long append(RegionInfo hri, WALKeyImpl key, WALEdit edits, boolean inMemstore)
559      throws IOException {
560    long txid =
561      stampSequenceIdAndPublishToRingBuffer(hri, key, edits, inMemstore, waitingConsumePayloads);
562    if (shouldScheduleConsumer()) {
563      consumeExecutor.execute(consumer);
564    }
565    return txid;
566  }
567
568  @Override
569  public void sync() throws IOException {
570    try (TraceScope scope = TraceUtil.createTrace("AsyncFSWAL.sync")) {
571      long txid = waitingConsumePayloads.next();
572      SyncFuture future;
573      try {
574        future = getSyncFuture(txid);
575        RingBufferTruck truck = waitingConsumePayloads.get(txid);
576        truck.load(future);
577      } finally {
578        waitingConsumePayloads.publish(txid);
579      }
580      if (shouldScheduleConsumer()) {
581        consumeExecutor.execute(consumer);
582      }
583      blockOnSync(future);
584    }
585  }
586
587  @Override
588  public void sync(long txid) throws IOException {
589    if (highestSyncedTxid.get() >= txid) {
590      return;
591    }
592    try (TraceScope scope = TraceUtil.createTrace("AsyncFSWAL.sync")) {
593      // here we do not use ring buffer sequence as txid
594      long sequence = waitingConsumePayloads.next();
595      SyncFuture future;
596      try {
597        future = getSyncFuture(txid);
598        RingBufferTruck truck = waitingConsumePayloads.get(sequence);
599        truck.load(future);
600      } finally {
601        waitingConsumePayloads.publish(sequence);
602      }
603      if (shouldScheduleConsumer()) {
604        consumeExecutor.execute(consumer);
605      }
606      blockOnSync(future);
607    }
608  }
609
610  @Override
611  protected AsyncWriter createWriterInstance(Path path) throws IOException {
612    return AsyncFSWALProvider.createAsyncWriter(conf, fs, path, false,
613        this.blocksize, eventLoopGroup, channelClass);
614  }
615
616  private void waitForSafePoint() {
617    consumeLock.lock();
618    try {
619      int currentEpochAndState = epochAndState;
620      if (writerBroken(currentEpochAndState) || this.writer == null) {
621        return;
622      }
623      consumerScheduled.set(true);
624      epochAndState = currentEpochAndState | 1;
625      readyForRolling = false;
626      consumeExecutor.execute(consumer);
627      while (!readyForRolling) {
628        readyForRollingCond.awaitUninterruptibly();
629      }
630    } finally {
631      consumeLock.unlock();
632    }
633  }
634
635  private long closeWriter() {
636    AsyncWriter oldWriter = this.writer;
637    if (oldWriter != null) {
638      long fileLength = oldWriter.getLength();
639      closeExecutor.execute(() -> {
640        try {
641          oldWriter.close();
642        } catch (IOException e) {
643          LOG.warn("close old writer failed", e);
644        }
645      });
646      return fileLength;
647    } else {
648      return 0L;
649    }
650  }
651
652  @Override
653  protected void doReplaceWriter(Path oldPath, Path newPath, AsyncWriter nextWriter)
654      throws IOException {
655    Preconditions.checkNotNull(nextWriter);
656    waitForSafePoint();
657    long oldFileLen = closeWriter();
658    logRollAndSetupWalProps(oldPath, newPath, oldFileLen);
659    this.writer = nextWriter;
660    if (nextWriter instanceof AsyncProtobufLogWriter) {
661      this.fsOut = ((AsyncProtobufLogWriter) nextWriter).getOutput();
662    }
663    this.fileLengthAtLastSync = nextWriter.getLength();
664    this.rollRequested = false;
665    this.highestProcessedAppendTxidAtLastSync = 0L;
666    consumeLock.lock();
667    try {
668      consumerScheduled.set(true);
669      int currentEpoch = epochAndState >>> 2;
670      int nextEpoch = currentEpoch == MAX_EPOCH ? 0 : currentEpoch + 1;
671      // set a new epoch and also clear waitingRoll and writerBroken
672      this.epochAndState = nextEpoch << 2;
673      consumeExecutor.execute(consumer);
674    } finally {
675      consumeLock.unlock();
676    }
677  }
678
679  @Override
680  protected void doShutdown() throws IOException {
681    waitForSafePoint();
682    closeWriter();
683    closeExecutor.shutdown();
684    try {
685      if (!closeExecutor.awaitTermination(waitOnShutdownInSeconds, TimeUnit.SECONDS)) {
686        LOG.error("We have waited " + waitOnShutdownInSeconds + " seconds but" +
687          " the close of async writer doesn't complete." +
688          "Please check the status of underlying filesystem" +
689          " or increase the wait time by the config \"" + ASYNC_WAL_WAIT_ON_SHUTDOWN_IN_SECONDS +
690          "\"");
691      }
692    } catch (InterruptedException e) {
693      LOG.error("The wait for close of async writer is interrupted");
694      Thread.currentThread().interrupt();
695    }
696    IOException error = new IOException("WAL has been closed");
697    long nextCursor = waitingConsumePayloadsGatingSequence.get() + 1;
698    // drain all the pending sync requests
699    for (long cursorBound = waitingConsumePayloads.getCursor(); nextCursor <= cursorBound;
700      nextCursor++) {
701      if (!waitingConsumePayloads.isPublished(nextCursor)) {
702        break;
703      }
704      RingBufferTruck truck = waitingConsumePayloads.get(nextCursor);
705      switch (truck.type()) {
706        case SYNC:
707          syncFutures.add(truck.unloadSync());
708          break;
709        default:
710          break;
711      }
712    }
713    // and fail them
714    syncFutures.forEach(f -> f.done(f.getTxid(), error));
715    if (!(consumeExecutor instanceof EventLoop)) {
716      consumeExecutor.shutdown();
717    }
718  }
719
720  @Override
721  protected void doAppend(AsyncWriter writer, FSWALEntry entry) {
722    writer.append(entry);
723  }
724
725  @Override
726  DatanodeInfo[] getPipeline() {
727    AsyncFSOutput output = this.fsOut;
728    return output != null ? output.getPipeline() : new DatanodeInfo[0];
729  }
730
731  @Override
732  int getLogReplication() {
733    return getPipeline().length;
734  }
735
736  @Override
737  protected boolean doCheckLogLowReplication() {
738    // not like FSHLog, AsyncFSOutput will fail immediately if there are errors writing to DNs, so
739    // typically there is no 'low replication' state, only a 'broken' state.
740    AsyncFSOutput output = this.fsOut;
741    return output != null && output.isBroken();
742  }
743}