View Javadoc

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