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