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