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.InterruptedIOException;
23  import java.io.OutputStream;
24  import java.lang.reflect.InvocationTargetException;
25  import java.lang.reflect.Method;
26  import java.net.URLEncoder;
27  import java.util.ArrayList;
28  import java.util.Arrays;
29  import java.util.Comparator;
30  import java.util.HashMap;
31  import java.util.List;
32  import java.util.Map;
33  import java.util.NavigableMap;
34  import java.util.TreeMap;
35  import java.util.UUID;
36  import java.util.concurrent.BlockingQueue;
37  import java.util.concurrent.ConcurrentHashMap;
38  import java.util.concurrent.ConcurrentSkipListMap;
39  import java.util.concurrent.CopyOnWriteArrayList;
40  import java.util.concurrent.CountDownLatch;
41  import java.util.concurrent.ExecutionException;
42  import java.util.concurrent.ExecutorService;
43  import java.util.concurrent.Executors;
44  import java.util.concurrent.LinkedBlockingQueue;
45  import java.util.concurrent.TimeUnit;
46  import java.util.concurrent.atomic.AtomicInteger;
47  import java.util.concurrent.atomic.AtomicLong;
48  import java.util.concurrent.locks.ReentrantLock;
49  
50  import org.apache.commons.logging.Log;
51  import org.apache.commons.logging.LogFactory;
52  import org.apache.hadoop.classification.InterfaceAudience;
53  import org.apache.hadoop.conf.Configuration;
54  import org.apache.hadoop.fs.FSDataOutputStream;
55  import org.apache.hadoop.fs.FileStatus;
56  import org.apache.hadoop.fs.FileSystem;
57  import org.apache.hadoop.fs.Path;
58  import org.apache.hadoop.fs.Syncable;
59  import org.apache.hadoop.hbase.HBaseConfiguration;
60  import org.apache.hadoop.hbase.HConstants;
61  import org.apache.hadoop.hbase.HRegionInfo;
62  import org.apache.hadoop.hbase.HTableDescriptor;
63  import org.apache.hadoop.hbase.KeyValue;
64  import org.apache.hadoop.hbase.TableName;
65  import org.apache.hadoop.hbase.util.Bytes;
66  import org.apache.hadoop.hbase.util.ClassSize;
67  import org.apache.hadoop.hbase.util.DrainBarrier;
68  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
69  import org.apache.hadoop.hbase.util.FSUtils;
70  import org.apache.hadoop.hbase.util.HasThread;
71  import org.apache.hadoop.hbase.util.Threads;
72  import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
73  import org.apache.hadoop.util.StringUtils;
74  import org.htrace.NullScope;
75  import org.htrace.Span;
76  import org.htrace.Trace;
77  import org.htrace.TraceScope;
78  
79  import com.google.common.annotations.VisibleForTesting;
80  import com.lmax.disruptor.BlockingWaitStrategy;
81  import com.lmax.disruptor.EventHandler;
82  import com.lmax.disruptor.ExceptionHandler;
83  import com.lmax.disruptor.LifecycleAware;
84  import com.lmax.disruptor.TimeoutException;
85  import com.lmax.disruptor.dsl.Disruptor;
86  import com.lmax.disruptor.dsl.ProducerType;
87  
88  /**
89   * Implementation of {@link HLog} to go against {@link FileSystem}; i.e. keep WALs in HDFS.
90   * Only one HLog/WAL is ever being written at a time.  When a WAL hits a configured maximum size,
91   * it is rolled.  This is done internal to the implementation.
92   *
93   * <p>As data is flushed from the MemStore to other on-disk structures (files sorted by
94   * key, hfiles), a WAL becomes obsolete. We can let go of all the log edits/entries for a given
95   * HRegion-sequence id.  A bunch of work in the below is done keeping account of these region
96   * sequence ids -- what is flushed out to hfiles, and what is yet in WAL and in memory only.
97   *
98   * <p>It is only practical to delete entire files. Thus, we delete an entire on-disk file
99   * <code>F</code> when all of the edits in <code>F</code> have a log-sequence-id that's older
100  * (smaller) than the most-recent flush.
101  *
102  * <p>To read an HLog, call {@link HLogFactory#createReader(org.apache.hadoop.fs.FileSystem,
103  * org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration)}.
104  */
105 @InterfaceAudience.Private
106 class FSHLog implements HLog, Syncable {
107   // IMPLEMENTATION NOTES:
108   //
109   // At the core is a ring buffer.  Our ring buffer is the LMAX Disruptor.  It tries to
110   // minimize synchronizations and volatile writes when multiple contending threads as is the case
111   // here appending and syncing on a single WAL.  The Disruptor is configured to handle multiple
112   // producers but it has one consumer only (the producers in HBase are IPC Handlers calling append
113   // and then sync).  The single consumer/writer pulls the appends and syncs off the ring buffer.
114   // When a handler calls sync, it is given back a future. The producer 'blocks' on the future so
115   // it does not return until the sync completes.  The future is passed over the ring buffer from
116   // the producer/handler to the consumer thread where it does its best to batch up the producer
117   // syncs so one WAL sync actually spans multiple producer sync invocations.  How well the
118   // batching works depends on the write rate; i.e. we tend to batch more in times of
119   // high writes/syncs.
120   //
121   // Calls to append now also wait until the append has been done on the consumer side of the
122   // disruptor.  We used to not wait but it makes the implemenation easier to grok if we have
123   // the region edit/sequence id after the append returns.
124   // 
125   // TODO: Handlers need to coordinate appending AND syncing.  Can we have the threads contend
126   // once only?  Probably hard given syncs take way longer than an append.
127   //
128   // The consumer threads pass the syncs off to multiple syncing threads in a round robin fashion
129   // to ensure we keep up back-to-back FS sync calls (FS sync calls are the long poll writing the
130   // WAL).  The consumer thread passes the futures to the sync threads for it to complete
131   // the futures when done.
132   //
133   // The 'sequence' in the below is the sequence of the append/sync on the ringbuffer.  It
134   // acts as a sort-of transaction id.  It is always incrementing.
135   //
136   // The RingBufferEventHandler class hosts the ring buffer consuming code.  The threads that
137   // do the actual FS sync are implementations of SyncRunner.  SafePointZigZagLatch is a
138   // synchronization class used to halt the consumer at a safe point --  just after all outstanding
139   // syncs and appends have completed -- so the log roller can swap the WAL out under it.
140 
141   static final Log LOG = LogFactory.getLog(FSHLog.class);
142 
143   private static final int DEFAULT_SLOW_SYNC_TIME_MS = 100; // in ms
144   
145   /**
146    * The nexus at which all incoming handlers meet.  Does appends and sync with an ordering.
147    * Appends and syncs are each put on the ring which means handlers need to
148    * smash up against the ring twice (can we make it once only? ... maybe not since time to append
149    * is so different from time to sync and sometimes we don't want to sync or we want to async
150    * the sync).  The ring is where we make sure of our ordering and it is also where we do
151    * batching up of handler sync calls.
152    */
153   private final Disruptor<RingBufferTruck> disruptor;
154 
155   /**
156    * An executorservice that runs the disrutpor AppendEventHandler append executor.
157    */
158   private final ExecutorService appendExecutor;
159 
160   /**
161    * This fellow is run by the above appendExecutor service but it is all about batching up appends
162    * and syncs; it may shutdown without cleaning out the last few appends or syncs.  To guard
163    * against this, keep a reference to this handler and do explicit close on way out to make sure
164    * all flushed out before we exit.
165    */
166   private final RingBufferEventHandler ringBufferEventHandler;
167 
168   /**
169    * Map of {@link SyncFuture}s keyed by Handler objects.  Used so we reuse SyncFutures.
170    * TODO: Reus FSWALEntry's rather than create them anew each time as we do SyncFutures here.
171    * TODO: Add a FSWalEntry and SyncFuture as thread locals on handlers rather than have them
172    * get them from this Map?
173    */
174   private final Map<Thread, SyncFuture> syncFuturesByHandler;
175 
176   private final FileSystem fs;
177   private final Path fullPathLogDir;
178   private final Path fullPathOldLogDir;
179   private final Configuration conf;
180   private final String logFilePrefix;
181 
182   /**
183    * The highest known outstanding unsync'd WALEdit sequence number where sequence number is the
184    * ring buffer sequence.  Maintained by the ring buffer consumer.
185    */
186   private volatile long highestUnsyncedSequence = -1;
187 
188   /**
189    * Updated to the ring buffer sequence of the last successful sync call.  This can be less than
190    * {@link #highestUnsyncedSequence} for case where we have an append where a sync has not yet
191    * come in for it.  Maintained by the syncing threads.
192    */
193   private final AtomicLong highestSyncedSequence = new AtomicLong(0);
194 
195   private WALCoprocessorHost coprocessorHost;
196 
197   /**
198    * FSDataOutputStream associated with the current SequenceFile.writer
199    */
200   private FSDataOutputStream hdfs_out;
201 
202   // All about log rolling if not enough replicas outstanding.
203 
204   // Minimum tolerable replicas, if the actual value is lower than it, rollWriter will be triggered
205   private final int minTolerableReplication;
206 
207   // DFSOutputStream.getNumCurrentReplicas method instance gotten via reflection.
208   private final Method getNumCurrentReplicas;
209   private final Method getPipeLine; // refers to DFSOutputStream.getPipeLine
210   private final int slowSyncNs;
211 
212   private final static Object [] NO_ARGS = new Object []{};
213 
214   // If live datanode count is lower than the default replicas value,
215   // RollWriter will be triggered in each sync(So the RollWriter will be
216   // triggered one by one in a short time). Using it as a workaround to slow
217   // down the roll frequency triggered by checkLowReplication().
218   private final AtomicInteger consecutiveLogRolls = new AtomicInteger(0);
219 
220   private final int lowReplicationRollLimit;
221 
222   // If consecutiveLogRolls is larger than lowReplicationRollLimit,
223   // then disable the rolling in checkLowReplication().
224   // Enable it if the replications recover.
225   private volatile boolean lowReplicationRollEnabled = true;
226 
227   /**
228    * Current log file.
229    */
230   volatile Writer writer;
231 
232   /** The barrier used to ensure that close() waits for all log rolls and flushes to finish. */
233   private final DrainBarrier closeBarrier = new DrainBarrier();
234 
235   /**
236    * This lock makes sure only one log roll runs at a time. Should not be taken while any other
237    * lock is held. We don't just use synchronized because that results in bogus and tedious
238    * findbugs warning when it thinks synchronized controls writer thread safety.  It is held when
239    * we are actually rolling the log.  It is checked when we are looking to see if we should roll
240    * the log or not.
241    */
242   private final ReentrantLock rollWriterLock = new ReentrantLock(true);
243 
244   // Listeners that are called on WAL events.
245   private final List<WALActionsListener> listeners =
246     new CopyOnWriteArrayList<WALActionsListener>();
247 
248   private volatile boolean closed = false;
249 
250   /**
251    * Set when this WAL is for meta only (we run a WAL for all regions except meta -- it has its
252    * own dedicated WAL).
253    */
254   private final boolean forMeta;
255 
256   // The timestamp (in ms) when the log file was created.
257   private final AtomicLong filenum = new AtomicLong(-1);
258 
259   // Number of transactions in the current Hlog.
260   private final AtomicInteger numEntries = new AtomicInteger(0);
261 
262   // If > than this size, roll the log.
263   private final long logrollsize;
264 
265   /**
266    * The total size of hlog
267    */
268   private AtomicLong totalLogSize = new AtomicLong(0);
269 
270   /**
271    * If WAL is enabled.
272    */
273   private final boolean enabled;
274 
275   /*
276    * If more than this many logs, force flush of oldest region to oldest edit
277    * goes to disk.  If too many and we crash, then will take forever replaying.
278    * Keep the number of logs tidy.
279    */
280   private final int maxLogs;
281 
282   /** Number of log close errors tolerated before we abort */
283   private final int closeErrorsTolerated;
284 
285   private final AtomicInteger closeErrorCount = new AtomicInteger();
286   private final MetricsWAL metrics;
287 
288   // Region sequence id accounting across flushes and for knowing when we can GC a WAL.  These
289   // sequence id numbers are by region and unrelated to the ring buffer sequence number accounting
290   // done above in failedSequence, highest sequence, etc.
291   /**
292    * This lock ties all operations on oldestFlushingRegionSequenceIds and
293    * oldestFlushedRegionSequenceIds Maps with the exception of append's putIfAbsent call into
294    * oldestUnflushedSeqNums. We use these Maps to find out the low bound regions sequence id, or
295    * to find regions  with old sequence ids to force flush; we are interested in old stuff not the
296    * new additions (TODO: IS THIS SAFE?  CHECK!).
297    */
298   private final Object regionSequenceIdLock = new Object();
299 
300   /**
301    * Map of encoded region names to their OLDEST -- i.e. their first, the longest-lived --
302    * sequence id in memstore. Note that this sequence id is the region sequence id.  This is not
303    * related to the id we use above for {@link #highestSyncedSequence} and
304    * {@link #highestUnsyncedSequence} which is the sequence from the disruptor ring buffer.
305    */
306   private final ConcurrentSkipListMap<byte [], Long> oldestUnflushedRegionSequenceIds =
307     new ConcurrentSkipListMap<byte [], Long>(Bytes.BYTES_COMPARATOR);
308 
309   /**
310    * Map of encoded region names to their lowest or OLDEST sequence/edit id in memstore currently
311    * being flushed out to hfiles. Entries are moved here from
312    * {@link #oldestUnflushedRegionSequenceIds} while the lock {@link #regionSequenceIdLock} is held
313    * (so movement between the Maps is atomic). This is not related to the id we use above for
314    * {@link #highestSyncedSequence} and {@link #highestUnsyncedSequence} which is the sequence from
315    * the disruptor ring buffer, an internal detail.
316    */
317   private final Map<byte[], Long> lowestFlushingRegionSequenceIds =
318     new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
319 
320  /**
321   * Map of region encoded names to the latest region sequence id.  Updated on each append of
322   * WALEdits to the WAL. We create one map for each WAL file at the time it is rolled.
323   * <p>When deciding whether to archive a WAL file, we compare the sequence IDs in this map to
324   * {@link #lowestFlushingRegionSequenceIds} and {@link #oldestUnflushedRegionSequenceIds}.
325   * See {@link FSHLog#areAllRegionsFlushed(Map, Map, Map)} for more info.
326   * <p>
327   * This map uses byte[] as the key, and uses reference equality. It works in our use case as we
328   * use {@link HRegionInfo#getEncodedNameAsBytes()} as keys. For a given region, it always returns
329   * the same array.
330   */
331   private Map<byte[], Long> highestRegionSequenceIds = new HashMap<byte[], Long>();
332 
333   /**
334    * WAL Comparator; it compares the timestamp (log filenum), present in the log file name.
335    */
336   public final Comparator<Path> LOG_NAME_COMPARATOR = new Comparator<Path>() {
337     @Override
338     public int compare(Path o1, Path o2) {
339       long t1 = getFileNumFromFileName(o1);
340       long t2 = getFileNumFromFileName(o2);
341       if (t1 == t2) return 0;
342       return (t1 > t2) ? 1 : -1;
343     }
344   };
345 
346   /**
347    * Map of wal log file to the latest sequence ids of all regions it has entries of.
348    * The map is sorted by the log file creation timestamp (contained in the log file name).
349    */
350   private NavigableMap<Path, Map<byte[], Long>> byWalRegionSequenceIds =
351     new ConcurrentSkipListMap<Path, Map<byte[], Long>>(LOG_NAME_COMPARATOR);
352 
353   /**
354    * Exception handler to pass the disruptor ringbuffer.  Same as native implementation only it
355    * logs using our logger instead of java native logger.
356    */
357   static class RingBufferExceptionHandler implements ExceptionHandler {
358     @Override
359     public void handleEventException(Throwable ex, long sequence, Object event) {
360       LOG.error("Sequence=" + sequence + ", event=" + event, ex);
361       throw new RuntimeException(ex);
362     }
363 
364     @Override
365     public void handleOnStartException(Throwable ex) {
366       LOG.error(ex);
367       throw new RuntimeException(ex);
368     }
369 
370     @Override
371     public void handleOnShutdownException(Throwable ex) {
372       LOG.error(ex);
373       throw new RuntimeException(ex);
374     }
375   }
376 
377   /**
378    * Constructor.
379    *
380    * @param fs filesystem handle
381    * @param root path for stored and archived hlogs
382    * @param logDir dir where hlogs are stored
383    * @param conf configuration to use
384    * @throws IOException
385    */
386   public FSHLog(final FileSystem fs, final Path root, final String logDir,
387     final Configuration conf)
388   throws IOException {
389     this(fs, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, false);
390   }
391 
392   /**
393    * Create an edit log at the given <code>dir</code> location.
394    *
395    * You should never have to load an existing log. If there is a log at
396    * startup, it should have already been processed and deleted by the time the
397    * HLog object is started up.
398    *
399    * @param fs filesystem handle
400    * @param rootDir path to where logs and oldlogs
401    * @param logDir dir where hlogs are stored
402    * @param oldLogDir dir where hlogs are archived
403    * @param conf configuration to use
404    * @param listeners Listeners on WAL events. Listeners passed here will
405    * be registered before we do anything else; e.g. the
406    * Constructor {@link #rollWriter()}.
407    * @param failIfLogDirExists If true IOException will be thrown if dir already exists.
408    * @param prefix should always be hostname and port in distributed env and
409    *        it will be URL encoded before being used.
410    *        If prefix is null, "hlog" will be used
411    * @param forMeta if this hlog is meant for meta updates
412    * @throws IOException
413    */
414   public FSHLog(final FileSystem fs, final Path rootDir, final String logDir,
415       final String oldLogDir, final Configuration conf,
416       final List<WALActionsListener> listeners,
417       final boolean failIfLogDirExists, final String prefix, boolean forMeta)
418   throws IOException {
419     super();
420     this.fs = fs;
421     this.fullPathLogDir = new Path(rootDir, logDir);
422     this.fullPathOldLogDir = new Path(rootDir, oldLogDir);
423     this.forMeta = forMeta;
424     this.conf = conf;
425 
426     // Register listeners.  TODO: Should this exist anymore?  We have CPs?
427     if (listeners != null) {
428       for (WALActionsListener i: listeners) {
429         registerWALActionsListener(i);
430       }
431     }
432 
433     // Get size to roll log at. Roll at 95% of HDFS block size so we avoid crossing HDFS blocks
434     // (it costs a little x'ing bocks)
435     long blocksize = this.conf.getLong("hbase.regionserver.hlog.blocksize",
436       FSUtils.getDefaultBlockSize(this.fs, this.fullPathLogDir));
437     this.logrollsize =
438       (long)(blocksize * conf.getFloat("hbase.regionserver.logroll.multiplier", 0.95f));
439 
440     this.maxLogs = conf.getInt("hbase.regionserver.maxlogs", 32);
441     this.minTolerableReplication = conf.getInt( "hbase.regionserver.hlog.tolerable.lowreplication",
442         FSUtils.getDefaultReplication(fs, this.fullPathLogDir));
443     this.lowReplicationRollLimit =
444       conf.getInt("hbase.regionserver.hlog.lowreplication.rolllimit", 5);
445     this.enabled = conf.getBoolean("hbase.regionserver.hlog.enabled", true);
446     this.closeErrorsTolerated = conf.getInt("hbase.regionserver.logroll.errors.tolerated", 0);
447     // If prefix is null||empty then just name it hlog
448     this.logFilePrefix =
449       prefix == null || prefix.isEmpty() ? "hlog" : URLEncoder.encode(prefix, "UTF8");
450     int maxHandlersCount = conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT, 200);
451 
452     LOG.info("WAL configuration: blocksize=" + StringUtils.byteDesc(blocksize) +
453       ", rollsize=" + StringUtils.byteDesc(this.logrollsize) +
454       ", enabled=" + this.enabled + ", prefix=" + this.logFilePrefix + ", logDir=" +
455       this.fullPathLogDir + ", oldLogDir=" + this.fullPathOldLogDir);
456 
457     boolean dirExists = false;
458     if (failIfLogDirExists && (dirExists = this.fs.exists(fullPathLogDir))) {
459       throw new IOException("Target HLog directory already exists: " + fullPathLogDir);
460     }
461     if (!dirExists && !fs.mkdirs(fullPathLogDir)) {
462       throw new IOException("Unable to mkdir " + fullPathLogDir);
463     }
464 
465     if (!fs.exists(this.fullPathOldLogDir)) {
466       if (!fs.mkdirs(this.fullPathOldLogDir)) {
467         throw new IOException("Unable to mkdir " + this.fullPathOldLogDir);
468       }
469     }
470 
471     // rollWriter sets this.hdfs_out if it can.
472     rollWriter();
473 
474     this.slowSyncNs =
475         1000000 * conf.getInt("hbase.regionserver.hlog.slowsync.ms",
476           DEFAULT_SLOW_SYNC_TIME_MS);
477     // handle the reflection necessary to call getNumCurrentReplicas(). TODO: Replace with
478     // HdfsDataOutputStream#getCurrentBlockReplication() and go without reflection.
479     this.getNumCurrentReplicas = getGetNumCurrentReplicas(this.hdfs_out);
480     this.getPipeLine = getGetPipeline(this.hdfs_out);
481 
482     this.coprocessorHost = new WALCoprocessorHost(this, conf);
483     this.metrics = new MetricsWAL();
484 
485     // This is the 'writer' -- a single threaded executor.  This single thread 'consumes' what is
486     // put on the ring buffer.
487     String hostingThreadName = Thread.currentThread().getName();
488     this.appendExecutor = Executors.
489       newSingleThreadExecutor(Threads.getNamedThreadFactory(hostingThreadName + ".append"));
490     // Preallocate objects to use on the ring buffer.  The way that appends and syncs work, we will
491     // be stuck and make no progress if the buffer is filled with appends only and there is no
492     // sync. If no sync, then the handlers will be outstanding just waiting on sync completion
493     // before they return.
494     final int preallocatedEventCount =
495       this.conf.getInt("hbase.regionserver.wal.disruptor.event.count", 1024 * 16);
496     // Using BlockingWaitStrategy.  Stuff that is going on here takes so long it makes no sense
497     // spinning as other strategies do.
498     this.disruptor =
499       new Disruptor<RingBufferTruck>(RingBufferTruck.EVENT_FACTORY, preallocatedEventCount,
500         this.appendExecutor, ProducerType.MULTI, new BlockingWaitStrategy());
501     // Advance the ring buffer sequence so that it starts from 1 instead of 0,
502     // because SyncFuture.NOT_DONE = 0.
503     this.disruptor.getRingBuffer().next();
504     this.ringBufferEventHandler =
505       new RingBufferEventHandler(conf.getInt("hbase.regionserver.hlog.syncer.count", 5),
506         maxHandlersCount);
507     this.disruptor.handleExceptionsWith(new RingBufferExceptionHandler());
508     this.disruptor.handleEventsWith(new RingBufferEventHandler [] {this.ringBufferEventHandler});
509     // Presize our map of SyncFutures by handler objects.
510     this.syncFuturesByHandler = new ConcurrentHashMap<Thread, SyncFuture>(maxHandlersCount);
511     // Starting up threads in constructor is a no no; Interface should have an init call.
512     this.disruptor.start();
513   }
514 
515   /**
516    * Find the 'getNumCurrentReplicas' on the passed <code>os</code> stream.
517    * @return Method or null.
518    */
519   private static Method getGetNumCurrentReplicas(final FSDataOutputStream os) {
520     // TODO: Remove all this and use the now publically available
521     // HdfsDataOutputStream#getCurrentBlockReplication()
522     Method m = null;
523     if (os != null) {
524       Class<? extends OutputStream> wrappedStreamClass = os.getWrappedStream().getClass();
525       try {
526         m = wrappedStreamClass.getDeclaredMethod("getNumCurrentReplicas", new Class<?>[] {});
527         m.setAccessible(true);
528       } catch (NoSuchMethodException e) {
529         LOG.info("FileSystem's output stream doesn't support getNumCurrentReplicas; " +
530          "HDFS-826 not available; fsOut=" + wrappedStreamClass.getName());
531       } catch (SecurityException e) {
532         LOG.info("No access to getNumCurrentReplicas on FileSystems's output stream; HDFS-826 " +
533           "not available; fsOut=" + wrappedStreamClass.getName(), e);
534         m = null; // could happen on setAccessible()
535       }
536     }
537     if (m != null) {
538       if (LOG.isTraceEnabled()) LOG.trace("Using getNumCurrentReplicas");
539     }
540     return m;
541   }
542 
543   @Override
544   public void registerWALActionsListener(final WALActionsListener listener) {
545     this.listeners.add(listener);
546   }
547 
548   @Override
549   public boolean unregisterWALActionsListener(final WALActionsListener listener) {
550     return this.listeners.remove(listener);
551   }
552 
553   @Override
554   public long getFilenum() {
555     return this.filenum.get();
556   }
557 
558   /**
559    * Method used internal to this class and for tests only.
560    * @return The wrapped stream our writer is using; its not the
561    * writer's 'out' FSDatoOutputStream but the stream that this 'out' wraps
562    * (In hdfs its an instance of DFSDataOutputStream).
563    *
564    * usage: see TestLogRolling.java
565    */
566   OutputStream getOutputStream() {
567     return this.hdfs_out.getWrappedStream();
568   }
569 
570   @Override
571   public byte [][] rollWriter() throws FailedLogCloseException, IOException {
572     return rollWriter(false);
573   }
574 
575   private Path getNewPath() throws IOException {
576     this.filenum.set(System.currentTimeMillis());
577     Path newPath = computeFilename();
578     while (fs.exists(newPath)) {
579       this.filenum.incrementAndGet();
580       newPath = computeFilename();
581     }
582     return newPath;
583   }
584 
585   Path getOldPath() {
586     long currentFilenum = this.filenum.get();
587     Path oldPath = null;
588     if (currentFilenum > 0) {
589       // ComputeFilename  will take care of meta hlog filename
590       oldPath = computeFilename(currentFilenum);
591     } // I presume if currentFilenum is <= 0, this is first file and null for oldPath if fine?
592     return oldPath;
593   }
594 
595   /**
596    * Tell listeners about pre log roll.
597    * @throws IOException 
598    */
599   private void tellListenersAboutPreLogRoll(final Path oldPath, final Path newPath)
600   throws IOException {
601     if (!this.listeners.isEmpty()) {
602       for (WALActionsListener i : this.listeners) {
603         i.preLogRoll(oldPath, newPath);
604       }
605     }
606   }
607 
608   /**
609    * Tell listeners about post log roll.
610    * @throws IOException 
611    */
612   private void tellListenersAboutPostLogRoll(final Path oldPath, final Path newPath)
613   throws IOException {
614     if (!this.listeners.isEmpty()) {
615       for (WALActionsListener i : this.listeners) {
616         i.postLogRoll(oldPath, newPath);
617       }
618     }
619   }
620 
621   /**
622    * Run a sync after opening to set up the pipeline.
623    * @param nextWriter
624    * @param startTimeNanos
625    */
626   private void preemptiveSync(final ProtobufLogWriter nextWriter) {
627     long startTimeNanos = System.nanoTime();
628     try {
629       nextWriter.sync();
630       postSync(System.nanoTime() - startTimeNanos, 0);
631     } catch (IOException e) {
632       // optimization failed, no need to abort here.
633       LOG.warn("pre-sync failed but an optimization so keep going", e);
634     }
635   }
636 
637   @Override
638   public byte [][] rollWriter(boolean force) throws FailedLogCloseException, IOException {
639     rollWriterLock.lock();
640     try {
641       // Return if nothing to flush.
642       if (!force && (this.writer != null && this.numEntries.get() <= 0)) return null;
643       byte [][] regionsToFlush = null;
644       if (this.closed) {
645         LOG.debug("HLog closed. Skipping rolling of writer");
646         return regionsToFlush;
647       }
648       if (!closeBarrier.beginOp()) {
649         LOG.debug("HLog closing. Skipping rolling of writer");
650         return regionsToFlush;
651       }
652       TraceScope scope = Trace.startSpan("FSHLog.rollWriter");
653       try {
654         Path oldPath = getOldPath();
655         Path newPath = getNewPath();
656         // Any exception from here on is catastrophic, non-recoverable so we currently abort.
657         FSHLog.Writer nextWriter = this.createWriterInstance(fs, newPath, conf);
658         FSDataOutputStream nextHdfsOut = null;
659         if (nextWriter instanceof ProtobufLogWriter) {
660           nextHdfsOut = ((ProtobufLogWriter)nextWriter).getStream();
661           // If a ProtobufLogWriter, go ahead and try and sync to force setup of pipeline.
662           // If this fails, we just keep going.... it is an optimization, not the end of the world.
663           preemptiveSync((ProtobufLogWriter)nextWriter);
664         }
665         tellListenersAboutPreLogRoll(oldPath, newPath);
666         // NewPath could be equal to oldPath if replaceWriter fails.
667         newPath = replaceWriter(oldPath, newPath, nextWriter, nextHdfsOut);
668         tellListenersAboutPostLogRoll(oldPath, newPath);
669         // Can we delete any of the old log files?
670         if (getNumRolledLogFiles() > 0) {
671           cleanOldLogs();
672           regionsToFlush = findRegionsToForceFlush();
673         }
674       } finally {
675         closeBarrier.endOp();
676         assert scope == NullScope.INSTANCE || !scope.isDetached();
677         scope.close();
678       }
679       return regionsToFlush;
680     } finally {
681       rollWriterLock.unlock();
682     }
683   }
684 
685   /**
686    * This method allows subclasses to inject different writers without having to
687    * extend other methods like rollWriter().
688    *
689    * @param fs
690    * @param path
691    * @param conf
692    * @return Writer instance
693    * @throws IOException
694    */
695   protected Writer createWriterInstance(final FileSystem fs, final Path path,
696       final Configuration conf) throws IOException {
697     if (forMeta) {
698       //TODO: set a higher replication for the hlog files (HBASE-6773)
699     }
700     return HLogFactory.createWALWriter(fs, path, conf);
701   }
702 
703   /**
704    * Archive old logs that could be archived: a log is eligible for archiving if all its WALEdits
705    * have been flushed to hfiles.
706    * <p>
707    * For each log file, it compares its region to sequenceId map
708    * (@link {@link FSHLog#highestRegionSequenceIds} with corresponding region entries in
709    * {@link FSHLog#lowestFlushingRegionSequenceIds} and
710    * {@link FSHLog#oldestUnflushedRegionSequenceIds}. If all the regions in the map are flushed
711    * past of their value, then the wal is eligible for archiving.
712    * @throws IOException
713    */
714   private void cleanOldLogs() throws IOException {
715     Map<byte[], Long> oldestFlushingSeqNumsLocal = null;
716     Map<byte[], Long> oldestUnflushedSeqNumsLocal = null;
717     List<Path> logsToArchive = new ArrayList<Path>();
718     // make a local copy so as to avoid locking when we iterate over these maps.
719     synchronized (regionSequenceIdLock) {
720       oldestFlushingSeqNumsLocal = new HashMap<byte[], Long>(this.lowestFlushingRegionSequenceIds);
721       oldestUnflushedSeqNumsLocal =
722         new HashMap<byte[], Long>(this.oldestUnflushedRegionSequenceIds);
723     }
724     for (Map.Entry<Path, Map<byte[], Long>> e : byWalRegionSequenceIds.entrySet()) {
725       // iterate over the log file.
726       Path log = e.getKey();
727       Map<byte[], Long> sequenceNums = e.getValue();
728       // iterate over the map for this log file, and tell whether it should be archive or not.
729       if (areAllRegionsFlushed(sequenceNums, oldestFlushingSeqNumsLocal,
730           oldestUnflushedSeqNumsLocal)) {
731         logsToArchive.add(log);
732         LOG.debug("WAL file ready for archiving " + log);
733       }
734     }
735     for (Path p : logsToArchive) {
736       this.totalLogSize.addAndGet(-this.fs.getFileStatus(p).getLen());
737       archiveLogFile(p);
738       this.byWalRegionSequenceIds.remove(p);
739     }
740   }
741 
742   /**
743    * Takes a region:sequenceId map for a WAL file, and checks whether the file can be archived.
744    * It compares the region entries present in the passed sequenceNums map with the local copy of
745    * {@link #oldestUnflushedRegionSequenceIds} and {@link #lowestFlushingRegionSequenceIds}. If,
746    * for all regions, the value is lesser than the minimum of values present in the
747    * oldestFlushing/UnflushedSeqNums, then the wal file is eligible for archiving.
748    * @param sequenceNums for a HLog, at the time when it was rolled.
749    * @param oldestFlushingMap
750    * @param oldestUnflushedMap
751    * @return true if wal is eligible for archiving, false otherwise.
752    */
753    static boolean areAllRegionsFlushed(Map<byte[], Long> sequenceNums,
754       Map<byte[], Long> oldestFlushingMap, Map<byte[], Long> oldestUnflushedMap) {
755     for (Map.Entry<byte[], Long> regionSeqIdEntry : sequenceNums.entrySet()) {
756       // find region entries in the flushing/unflushed map. If there is no entry, it meansj
757       // a region doesn't have any unflushed entry.
758       long oldestFlushing = oldestFlushingMap.containsKey(regionSeqIdEntry.getKey()) ?
759           oldestFlushingMap.get(regionSeqIdEntry.getKey()) : Long.MAX_VALUE;
760       long oldestUnFlushed = oldestUnflushedMap.containsKey(regionSeqIdEntry.getKey()) ?
761           oldestUnflushedMap.get(regionSeqIdEntry.getKey()) : Long.MAX_VALUE;
762           // do a minimum to be sure to contain oldest sequence Id
763       long minSeqNum = Math.min(oldestFlushing, oldestUnFlushed);
764       if (minSeqNum <= regionSeqIdEntry.getValue()) return false;// can't archive
765     }
766     return true;
767   }
768 
769   /**
770    * Iterates over the given map of regions, and compares their sequence numbers with corresponding
771    * entries in {@link #oldestUnflushedRegionSequenceIds}. If the sequence number is greater or
772    * equal, the region is eligible to flush, otherwise, there is no benefit to flush (from the
773    * perspective of passed regionsSequenceNums map), because the region has already flushed the
774    * entries present in the WAL file for which this method is called for (typically, the oldest
775    * wal file).
776    * @param regionsSequenceNums
777    * @return regions which should be flushed (whose sequence numbers are larger than their
778    * corresponding un-flushed entries.
779    */
780   private byte[][] findEligibleMemstoresToFlush(Map<byte[], Long> regionsSequenceNums) {
781     List<byte[]> regionsToFlush = null;
782     // Keeping the old behavior of iterating unflushedSeqNums under oldestSeqNumsLock.
783     synchronized (regionSequenceIdLock) {
784       for (Map.Entry<byte[], Long> e : regionsSequenceNums.entrySet()) {
785         Long unFlushedVal = this.oldestUnflushedRegionSequenceIds.get(e.getKey());
786         if (unFlushedVal != null && unFlushedVal <= e.getValue()) {
787           if (regionsToFlush == null) regionsToFlush = new ArrayList<byte[]>();
788           regionsToFlush.add(e.getKey());
789         }
790       }
791     }
792     return regionsToFlush == null ? null : regionsToFlush
793         .toArray(new byte[][] { HConstants.EMPTY_BYTE_ARRAY });
794   }
795 
796   /**
797    * If the number of un-archived WAL files is greater than maximum allowed, it checks
798    * the first (oldest) WAL file, and returns the regions which should be flushed so that it could
799    * be archived.
800    * @return regions to flush in order to archive oldest wal file.
801    * @throws IOException
802    */
803   byte[][] findRegionsToForceFlush() throws IOException {
804     byte [][] regions = null;
805     int logCount = getNumRolledLogFiles();
806     if (logCount > this.maxLogs && logCount > 0) {
807       Map.Entry<Path, Map<byte[], Long>> firstWALEntry =
808         this.byWalRegionSequenceIds.firstEntry();
809       regions = findEligibleMemstoresToFlush(firstWALEntry.getValue());
810     }
811     if (regions != null) {
812       StringBuilder sb = new StringBuilder();
813       for (int i = 0; i < regions.length; i++) {
814         if (i > 0) sb.append(", ");
815         sb.append(Bytes.toStringBinary(regions[i]));
816       }
817       LOG.info("Too many hlogs: logs=" + logCount + ", maxlogs=" +
818          this.maxLogs + "; forcing flush of " + regions.length + " regions(s): " +
819          sb.toString());
820     }
821     return regions;
822   }
823 
824   /**
825    * Cleans up current writer closing it and then puts in place the passed in
826    * <code>nextWriter</code>
827    * 
828    * @param oldPath
829    * @param newPath
830    * @param nextWriter
831    * @param nextHdfsOut
832    * @return <code>newPath</code>
833    * @throws IOException
834    */
835   Path replaceWriter(final Path oldPath, final Path newPath, FSHLog.Writer nextWriter,
836       final FSDataOutputStream nextHdfsOut)
837   throws IOException {
838     // Ask the ring buffer writer to pause at a safe point.  Once we do this, the writer
839     // thread will eventually pause. An error hereafter needs to release the writer thread
840     // regardless -- hence the finally block below.  Note, this method is called from the FSHLog
841     // constructor BEFORE the ring buffer is set running so it is null on first time through
842     // here; allow for that.
843     SyncFuture syncFuture = null;
844     SafePointZigZagLatch zigzagLatch = (this.ringBufferEventHandler == null)?
845       null: this.ringBufferEventHandler.attainSafePoint();
846     TraceScope scope = Trace.startSpan("FSHFile.replaceWriter");
847     try {
848       // Wait on the safe point to be achieved.  Send in a sync in case nothing has hit the
849       // ring buffer between the above notification of writer that we want it to go to
850       // 'safe point' and then here where we are waiting on it to attain safe point.  Use
851       // 'sendSync' instead of 'sync' because we do not want this thread to block waiting on it
852       // to come back.  Cleanup this syncFuture down below after we are ready to run again.
853       try {
854         if (zigzagLatch != null) {
855           Trace.addTimelineAnnotation("awaiting safepoint");
856           syncFuture = zigzagLatch.waitSafePoint(publishSyncOnRingBuffer());
857         }
858       } catch (FailedSyncBeforeLogCloseException e) {
859         if (isUnflushedEntries()) throw e;
860         // Else, let is pass through to the close.
861         LOG.warn("Failed last sync but no outstanding unsync edits so falling through to close; " +
862           e.getMessage());
863       }
864 
865       // It is at the safe point.  Swap out writer from under the blocked writer thread.
866       // TODO: This is close is inline with critical section.  Should happen in background?
867       try {
868         if (this.writer != null) {
869           Trace.addTimelineAnnotation("closing writer");
870           this.writer.close();
871           Trace.addTimelineAnnotation("writer closed");
872         }
873         this.closeErrorCount.set(0);
874       } catch (IOException ioe) {
875         int errors = closeErrorCount.incrementAndGet();
876         if (!isUnflushedEntries() && (errors <= this.closeErrorsTolerated)) {
877           LOG.warn("Riding over failed WAL close of " + oldPath + ", cause=\"" +
878             ioe.getMessage() + "\", errors=" + errors +
879             "; THIS FILE WAS NOT CLOSED BUT ALL EDITS SYNCED SO SHOULD BE OK");
880         } else {
881           throw ioe;
882         }
883       }
884       this.writer = nextWriter;
885       this.hdfs_out = nextHdfsOut;
886       int oldNumEntries = this.numEntries.get();
887       this.numEntries.set(0);
888       if (oldPath != null) {
889         this.byWalRegionSequenceIds.put(oldPath, this.highestRegionSequenceIds);
890         this.highestRegionSequenceIds = new HashMap<byte[], Long>();
891         long oldFileLen = this.fs.getFileStatus(oldPath).getLen();
892         this.totalLogSize.addAndGet(oldFileLen);
893         LOG.info("Rolled WAL " + FSUtils.getPath(oldPath) + " with entries=" + oldNumEntries +
894           ", filesize=" + StringUtils.byteDesc(oldFileLen) + "; new WAL " +
895           FSUtils.getPath(newPath));
896       } else {
897         LOG.info("New WAL " + FSUtils.getPath(newPath));
898       }
899     } catch (InterruptedException ie) {
900       // Perpetuate the interrupt
901       Thread.currentThread().interrupt();
902     } catch (IOException e) {
903       long count = getUnflushedEntriesCount();
904       LOG.error("Failed close of HLog writer " + oldPath + ", unflushedEntries=" + count, e);
905       throw new FailedLogCloseException(oldPath + ", unflushedEntries=" + count, e);
906     } finally {
907       try {
908         // Let the writer thread go regardless, whether error or not.
909         if (zigzagLatch != null) {
910           zigzagLatch.releaseSafePoint();
911           // It will be null if we failed our wait on safe point above.
912           if (syncFuture != null) blockOnSync(syncFuture);
913         }
914       } finally {
915         scope.close();
916       }
917     }
918     return newPath;
919   }
920 
921   long getUnflushedEntriesCount() {
922     long highestSynced = this.highestSyncedSequence.get();
923     return highestSynced > this.highestUnsyncedSequence?
924       0: this.highestUnsyncedSequence - highestSynced;
925   }
926 
927   boolean isUnflushedEntries() {
928     return getUnflushedEntriesCount() > 0;
929   }
930 
931   private void archiveLogFile(final Path p) throws IOException {
932     Path newPath = getHLogArchivePath(this.fullPathOldLogDir, p);
933     // Tell our listeners that a log is going to be archived.
934     if (!this.listeners.isEmpty()) {
935       for (WALActionsListener i : this.listeners) {
936         i.preLogArchive(p, newPath);
937       }
938     }
939     if (!FSUtils.renameAndSetModifyTime(this.fs, p, newPath)) {
940       throw new IOException("Unable to rename " + p + " to " + newPath);
941     }
942     // Tell our listeners that a log has been archived.
943     if (!this.listeners.isEmpty()) {
944       for (WALActionsListener i : this.listeners) {
945         i.postLogArchive(p, newPath);
946       }
947     }
948   }
949 
950   /**
951    * This is a convenience method that computes a new filename with a given
952    * file-number.
953    * @param filenum to use
954    * @return Path
955    */
956   protected Path computeFilename(final long filenum) {
957     this.filenum.set(filenum);
958     return computeFilename();
959   }
960 
961   /**
962    * This is a convenience method that computes a new filename with a given
963    * using the current HLog file-number
964    * @return Path
965    */
966   protected Path computeFilename() {
967     if (this.filenum.get() < 0) {
968       throw new RuntimeException("hlog file number can't be < 0");
969     }
970     String child = logFilePrefix + "." + filenum;
971     if (forMeta) {
972       child += HLog.META_HLOG_FILE_EXTN;
973     }
974     return new Path(fullPathLogDir, child);
975   }
976 
977 /**
978  * A log file has a creation timestamp (in ms) in its file name ({@link #filenum}.
979  * This helper method returns the creation timestamp from a given log file.
980  * It extracts the timestamp assuming the filename is created with the
981  * {@link #computeFilename(long filenum)} method.
982  * @param fileName
983  * @return timestamp, as in the log file name.
984  */
985   protected long getFileNumFromFileName(Path fileName) {
986     if (fileName == null) throw new IllegalArgumentException("file name can't be null");
987     // The path should start with dir/<prefix>.
988     String prefixPathStr = new Path(fullPathLogDir, logFilePrefix + ".").toString();
989     if (!fileName.toString().startsWith(prefixPathStr)) {
990       throw new IllegalArgumentException("The log file " + fileName + " doesn't belong to" +
991       		" this regionserver " + prefixPathStr);
992     }
993     String chompedPath = fileName.toString().substring(prefixPathStr.length());
994     if (forMeta) chompedPath = chompedPath.substring(0, chompedPath.indexOf(META_HLOG_FILE_EXTN));
995     return Long.parseLong(chompedPath);
996   }
997 
998   @Override
999   public void closeAndDelete() throws IOException {
1000     close();
1001     if (!fs.exists(this.fullPathLogDir)) return;
1002     FileStatus[] files = fs.listStatus(this.fullPathLogDir);
1003     if (files != null) {
1004       for(FileStatus file : files) {
1005 
1006         Path p = getHLogArchivePath(this.fullPathOldLogDir, file.getPath());
1007         // Tell our listeners that a log is going to be archived.
1008         if (!this.listeners.isEmpty()) {
1009           for (WALActionsListener i : this.listeners) {
1010             i.preLogArchive(file.getPath(), p);
1011           }
1012         }
1013 
1014         if (!FSUtils.renameAndSetModifyTime(fs, file.getPath(), p)) {
1015           throw new IOException("Unable to rename " + file.getPath() + " to " + p);
1016         }
1017         // Tell our listeners that a log was archived.
1018         if (!this.listeners.isEmpty()) {
1019           for (WALActionsListener i : this.listeners) {
1020             i.postLogArchive(file.getPath(), p);
1021           }
1022         }
1023       }
1024       LOG.debug("Moved " + files.length + " WAL file(s) to " +
1025         FSUtils.getPath(this.fullPathOldLogDir));
1026     }
1027     if (!fs.delete(fullPathLogDir, true)) {
1028       LOG.info("Unable to delete " + fullPathLogDir);
1029     }
1030   }
1031 
1032   @Override
1033   public void close() throws IOException {
1034     if (this.closed) return;
1035     try {
1036       // Prevent all further flushing and rolling.
1037       closeBarrier.stopAndDrainOps();
1038     } catch (InterruptedException e) {
1039       LOG.error("Exception while waiting for cache flushes and log rolls", e);
1040       Thread.currentThread().interrupt();
1041     }
1042 
1043     // Shutdown the disruptor.  Will stop after all entries have been processed.  Make sure we have
1044     // stopped incoming appends before calling this else it will not shutdown.  We are
1045     // conservative below waiting a long time and if not elapsed, then halting.
1046     if (this.disruptor != null) {
1047       long timeoutms = conf.getLong("hbase.wal.disruptor.shutdown.timeout.ms", 60000);
1048       try {
1049         this.disruptor.shutdown(timeoutms, TimeUnit.MILLISECONDS);
1050       } catch (TimeoutException e) {
1051         LOG.warn("Timed out bringing down disruptor after " + timeoutms + "ms; forcing halt " +
1052           "(It is a problem if this is NOT an ABORT! -- DATALOSS!!!!)");
1053         this.disruptor.halt();
1054         this.disruptor.shutdown();
1055       }
1056     }
1057     // With disruptor down, this is safe to let go.
1058     if (this.appendExecutor !=  null) this.appendExecutor.shutdown();
1059 
1060     // Tell our listeners that the log is closing
1061     if (!this.listeners.isEmpty()) {
1062       for (WALActionsListener i : this.listeners) {
1063         i.logCloseRequested();
1064       }
1065     }
1066     this.closed = true;
1067     if (LOG.isDebugEnabled()) {
1068       LOG.debug("Closing WAL writer in " + this.fullPathLogDir.toString());
1069     }
1070     if (this.writer != null) {
1071       this.writer.close();
1072       this.writer = null;
1073     }
1074   }
1075 
1076   /**
1077    * @param now
1078    * @param encodedRegionName Encoded name of the region as returned by
1079    * <code>HRegionInfo#getEncodedNameAsBytes()</code>.
1080    * @param tableName
1081    * @param clusterIds that have consumed the change
1082    * @return New log key.
1083    */
1084   protected HLogKey makeKey(byte[] encodedRegionName, TableName tableName, long seqnum,
1085       long now, List<UUID> clusterIds, long nonceGroup, long nonce) {
1086     return new HLogKey(encodedRegionName, tableName, seqnum, now, clusterIds, nonceGroup, nonce);
1087   }
1088   
1089   @Override
1090   @VisibleForTesting
1091   public void append(HRegionInfo info, TableName tableName, WALEdit edits,
1092     final long now, HTableDescriptor htd, AtomicLong sequenceId)
1093   throws IOException {
1094     HLogKey logKey = new HLogKey(info.getEncodedNameAsBytes(), tableName, now);
1095     append(htd, info, logKey, edits, sequenceId, true, true, null);
1096   }
1097 
1098   @Override
1099   public long appendNoSync(final HRegionInfo info, TableName tableName, WALEdit edits,
1100       List<UUID> clusterIds, final long now, HTableDescriptor htd, AtomicLong sequenceId,
1101       boolean inMemstore, long nonceGroup, long nonce) throws IOException {
1102     HLogKey logKey =
1103       new HLogKey(info.getEncodedNameAsBytes(), tableName, now, clusterIds, nonceGroup, nonce);
1104     return append(htd, info, logKey, edits, sequenceId, false, inMemstore, null);
1105   }
1106 
1107   @Override
1108   public long appendNoSync(final HTableDescriptor htd, final HRegionInfo info, final HLogKey key,
1109       final WALEdit edits, final AtomicLong sequenceId, final boolean inMemstore, 
1110       final List<KeyValue> memstoreKVs)
1111   throws IOException {
1112     return append(htd, info, key, edits, sequenceId, false, inMemstore, memstoreKVs);
1113   }
1114 
1115   /**
1116    * Append a set of edits to the log. Log edits are keyed by (encoded) regionName, rowname, and
1117    * log-sequence-id.
1118    * @param key
1119    * @param edits
1120    * @param htd This comes in here just so it is available on a pre append for replications.  Get
1121    * rid of it.  It is kinda crazy this comes in here when we have tablename and regioninfo.
1122    * Replication gets its scope from the HTD.
1123    * @param hri region info
1124    * @param sync shall we sync after we call the append?
1125    * @param inMemstore
1126    * @param sequenceId The region sequence id reference.
1127    * @param memstoreKVs
1128    * @return txid of this transaction or if nothing to do, the last txid
1129    * @throws IOException
1130    */
1131   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NP_NULL_ON_SOME_PATH_EXCEPTION",
1132       justification="Will never be null")
1133   private long append(HTableDescriptor htd, final HRegionInfo hri, final HLogKey key,
1134       WALEdit edits, AtomicLong sequenceId, boolean sync, boolean inMemstore, 
1135       List<KeyValue> memstoreKVs)
1136   throws IOException {
1137     if (!this.enabled) return this.highestUnsyncedSequence;
1138     if (this.closed) throw new IOException("Cannot append; log is closed");
1139     // Make a trace scope for the append.  It is closed on other side of the ring buffer by the
1140     // single consuming thread.  Don't have to worry about it.
1141     TraceScope scope = Trace.startSpan("FSHLog.append");
1142 
1143     // This is crazy how much it takes to make an edit.  Do we need all this stuff!!!!????  We need
1144     // all this to make a key and then below to append the edit, we need to carry htd, info,
1145     // etc. all over the ring buffer.
1146     FSWALEntry entry = null;
1147     long sequence = this.disruptor.getRingBuffer().next();
1148     try {
1149       RingBufferTruck truck = this.disruptor.getRingBuffer().get(sequence);
1150       // Construction of FSWALEntry sets a latch.  The latch is thrown just after we stamp the
1151       // edit with its edit/sequence id.  The below entry.getRegionSequenceId will wait on the
1152       // latch to be thrown.  TODO: reuse FSWALEntry as we do SyncFuture rather create per append.
1153       entry = new FSWALEntry(sequence, key, edits, sequenceId, inMemstore, htd, hri, memstoreKVs);
1154       truck.loadPayload(entry, scope.detach());
1155     } finally {
1156       this.disruptor.getRingBuffer().publish(sequence);
1157     }
1158     // doSync is set in tests.  Usually we arrive in here via appendNoSync w/ the sync called after
1159     // all edits on a handler have been added.
1160     if (sync) sync(sequence);
1161     return sequence;
1162   }
1163 
1164   /**
1165    * Thread to runs the hdfs sync call. This call takes a while to complete.  This is the longest
1166    * pole adding edits to the WAL and this must complete to be sure all edits persisted.  We run
1167    * multiple threads sync'ng rather than one that just syncs in series so we have better
1168    * latencies; otherwise, an edit that arrived just after a sync started, might have to wait
1169    * almost the length of two sync invocations before it is marked done.
1170    * <p>When the sync completes, it marks all the passed in futures done.  On the other end of the
1171    * sync future is a blocked thread, usually a regionserver Handler.  There may be more than one
1172    * future passed in the case where a few threads arrive at about the same time and all invoke
1173    * 'sync'.  In this case we'll batch up the invocations and run one filesystem sync only for a
1174    * batch of Handler sync invocations.  Do not confuse these Handler SyncFutures with the futures
1175    * an ExecutorService returns when you call submit. We have no use for these in this model. These
1176    * SyncFutures are 'artificial', something to hold the Handler until the filesystem sync
1177    * completes.
1178    */
1179   private class SyncRunner extends HasThread {
1180     private volatile long sequence;
1181     private final BlockingQueue<SyncFuture> syncFutures;
1182  
1183     /**
1184      * UPDATE! 
1185      * @param syncs the batch of calls to sync that arrived as this thread was starting; when done,
1186      * we will put the result of the actual hdfs sync call as the result.
1187      * @param sequence The sequence number on the ring buffer when this thread was set running.
1188      * If this actual writer sync completes then all appends up this point have been
1189      * flushed/synced/pushed to datanodes.  If we fail, then the passed in <code>syncs</code>
1190      * futures will return the exception to their clients; some of the edits may have made it out
1191      * to data nodes but we will report all that were part of this session as failed.
1192      */
1193     SyncRunner(final String name, final int maxHandlersCount) {
1194       super(name);
1195       // LinkedBlockingQueue because of
1196       // http://www.javacodegeeks.com/2010/09/java-best-practices-queue-battle-and.html
1197       // Could use other blockingqueues here or concurrent queues.
1198       this.syncFutures = new LinkedBlockingQueue<SyncFuture>(maxHandlersCount);
1199     }
1200 
1201     void offer(final long sequence, final SyncFuture [] syncFutures, final int syncFutureCount) {
1202       // Set sequence first because the add to the queue will wake the thread if sleeping.
1203       this.sequence = sequence;
1204       for (int i = 0; i < syncFutureCount; i++) this.syncFutures.add(syncFutures[i]);
1205     }
1206 
1207     /**
1208      * Release the passed <code>syncFuture</code>
1209      * @param syncFuture
1210      * @param currentSequence
1211      * @param t
1212      * @return Returns 1.
1213      */
1214     private int releaseSyncFuture(final SyncFuture syncFuture, final long currentSequence,
1215         final Throwable t) {
1216       if (!syncFuture.done(currentSequence, t)) throw new IllegalStateException();
1217       // This function releases one sync future only.
1218       return 1;
1219     }
1220  
1221     /**
1222      * Release all SyncFutures whose sequence is <= <code>currentSequence</code>.
1223      * @param currentSequence
1224      * @param t May be non-null if we are processing SyncFutures because an exception was thrown.
1225      * @return Count of SyncFutures we let go.
1226      */
1227     private int releaseSyncFutures(final long currentSequence, final Throwable t) {
1228       int syncCount = 0;
1229       for (SyncFuture syncFuture; (syncFuture = this.syncFutures.peek()) != null;) {
1230         if (syncFuture.getRingBufferSequence() > currentSequence) break;
1231         releaseSyncFuture(syncFuture, currentSequence, t);
1232         if (!this.syncFutures.remove(syncFuture)) {
1233           throw new IllegalStateException(syncFuture.toString());
1234         }
1235         syncCount++;
1236       }
1237       return syncCount;
1238     }
1239 
1240     /**
1241      * @param sequence The sequence we ran the filesystem sync against.
1242      * @return Current highest synced sequence.
1243      */
1244     private long updateHighestSyncedSequence(long sequence) {
1245       long currentHighestSyncedSequence;
1246       // Set the highestSyncedSequence IFF our current sequence id is the 'highest'.
1247       do {
1248         currentHighestSyncedSequence = highestSyncedSequence.get();
1249         if (currentHighestSyncedSequence >= sequence) {
1250           // Set the sync number to current highwater mark; might be able to let go more
1251           // queued sync futures
1252           sequence = currentHighestSyncedSequence;
1253           break;
1254         }
1255       } while (!highestSyncedSequence.compareAndSet(currentHighestSyncedSequence, sequence));
1256       return sequence;
1257     }
1258 
1259     public void run() {
1260       long currentSequence;
1261       while (!isInterrupted()) {
1262         int syncCount = 0;
1263         SyncFuture takeSyncFuture;
1264         try {
1265           while (true) {
1266             // We have to process what we 'take' from the queue
1267             takeSyncFuture = this.syncFutures.take();
1268             currentSequence = this.sequence;
1269             long syncFutureSequence = takeSyncFuture.getRingBufferSequence();
1270             if (syncFutureSequence > currentSequence) {
1271               throw new IllegalStateException("currentSequence=" + syncFutureSequence +
1272                 ", syncFutureSequence=" + syncFutureSequence);
1273             }
1274             // See if we can process any syncfutures BEFORE we go sync.
1275             long currentHighestSyncedSequence = highestSyncedSequence.get();
1276             if (currentSequence < currentHighestSyncedSequence) {
1277               syncCount += releaseSyncFuture(takeSyncFuture, currentHighestSyncedSequence, null);
1278               // Done with the 'take'.  Go around again and do a new 'take'.
1279               continue;
1280             }
1281             break;
1282           }
1283           // I got something.  Lets run.  Save off current sequence number in case it changes
1284           // while we run.
1285           TraceScope scope = Trace.continueSpan(takeSyncFuture.getSpan());
1286           long start = System.nanoTime();
1287           Throwable t = null;
1288           try {
1289             Trace.addTimelineAnnotation("syncing writer");
1290             writer.sync();
1291             Trace.addTimelineAnnotation("writer synced");
1292             currentSequence = updateHighestSyncedSequence(currentSequence);
1293           } catch (IOException e) {
1294             LOG.error("Error syncing, request close of hlog ", e);
1295             t = e;
1296           } catch (Exception e) {
1297             LOG.warn("UNEXPECTED", e);
1298             t = e;
1299           } finally {
1300             // reattach the span to the future before releasing.
1301             takeSyncFuture.setSpan(scope.detach());
1302             // First release what we 'took' from the queue.
1303             syncCount += releaseSyncFuture(takeSyncFuture, currentSequence, t);
1304             // Can we release other syncs?
1305             syncCount += releaseSyncFutures(currentSequence, t);
1306             if (t != null) {
1307               requestLogRoll();
1308             } else checkLogRoll();
1309           }
1310           postSync(System.nanoTime() - start, syncCount);
1311         } catch (InterruptedException e) {
1312           // Presume legit interrupt.
1313           Thread.currentThread().interrupt();
1314         } catch (Throwable t) {
1315           LOG.warn("UNEXPECTED, continuing", t);
1316         }
1317       }
1318     }
1319   }
1320 
1321   /**
1322    * Schedule a log roll if needed.
1323    */
1324   void checkLogRoll() {
1325     // Will return immediately if we are in the middle of a WAL log roll currently.
1326     if (!rollWriterLock.tryLock()) return;
1327     boolean lowReplication;
1328     try {
1329       lowReplication = checkLowReplication();
1330     } finally {
1331       rollWriterLock.unlock();
1332     }
1333     try {
1334       if (lowReplication || writer != null && writer.getLength() > logrollsize) requestLogRoll();
1335     } catch (IOException e) {
1336       LOG.warn("Writer.getLength() failed; continuing", e);
1337     }
1338   }
1339 
1340   /*
1341    * @return true if number of replicas for the WAL is lower than threshold
1342    */
1343   private boolean checkLowReplication() {
1344     boolean logRollNeeded = false;
1345     // if the number of replicas in HDFS has fallen below the configured
1346     // value, then roll logs.
1347     try {
1348       int numCurrentReplicas = getLogReplication();
1349       if (numCurrentReplicas != 0 && numCurrentReplicas < this.minTolerableReplication) {
1350         if (this.lowReplicationRollEnabled) {
1351           if (this.consecutiveLogRolls.get() < this.lowReplicationRollLimit) {
1352             LOG.warn("HDFS pipeline error detected. " + "Found "
1353                 + numCurrentReplicas + " replicas but expecting no less than "
1354                 + this.minTolerableReplication + " replicas. "
1355                 + " Requesting close of hlog.");
1356             logRollNeeded = true;
1357             // If rollWriter is requested, increase consecutiveLogRolls. Once it
1358             // is larger than lowReplicationRollLimit, disable the
1359             // LowReplication-Roller
1360             this.consecutiveLogRolls.getAndIncrement();
1361           } else {
1362             LOG.warn("Too many consecutive RollWriter requests, it's a sign of "
1363                 + "the total number of live datanodes is lower than the tolerable replicas.");
1364             this.consecutiveLogRolls.set(0);
1365             this.lowReplicationRollEnabled = false;
1366           }
1367         }
1368       } else if (numCurrentReplicas >= this.minTolerableReplication) {
1369         if (!this.lowReplicationRollEnabled) {
1370           // The new writer's log replicas is always the default value.
1371           // So we should not enable LowReplication-Roller. If numEntries
1372           // is lower than or equals 1, we consider it as a new writer.
1373           if (this.numEntries.get() <= 1) {
1374             return logRollNeeded;
1375           }
1376           // Once the live datanode number and the replicas return to normal,
1377           // enable the LowReplication-Roller.
1378           this.lowReplicationRollEnabled = true;
1379           LOG.info("LowReplication-Roller was enabled.");
1380         }
1381       }
1382     } catch (Exception e) {
1383       LOG.warn("Unable to invoke DFSOutputStream.getNumCurrentReplicas" + e +
1384         " still proceeding ahead...");
1385     }
1386     return logRollNeeded;
1387   }
1388 
1389   private SyncFuture publishSyncOnRingBuffer() {
1390     return publishSyncOnRingBuffer(null);
1391   }
1392 
1393   private SyncFuture publishSyncOnRingBuffer(Span span) {
1394     long sequence = this.disruptor.getRingBuffer().next();
1395     SyncFuture syncFuture = getSyncFuture(sequence, span);
1396     try {
1397       RingBufferTruck truck = this.disruptor.getRingBuffer().get(sequence);
1398       truck.loadPayload(syncFuture);
1399     } finally {
1400       this.disruptor.getRingBuffer().publish(sequence);
1401     }
1402     return syncFuture;
1403   }
1404 
1405   // Sync all known transactions
1406   private Span publishSyncThenBlockOnCompletion(Span span) throws IOException {
1407     return blockOnSync(publishSyncOnRingBuffer(span));
1408   }
1409 
1410   private Span blockOnSync(final SyncFuture syncFuture) throws IOException {
1411     // Now we have published the ringbuffer, halt the current thread until we get an answer back.
1412     try {
1413       syncFuture.get();
1414       return syncFuture.getSpan();
1415     } catch (InterruptedException ie) {
1416       LOG.warn("Interrupted", ie);
1417       throw convertInterruptedExceptionToIOException(ie);
1418     } catch (ExecutionException e) {
1419       throw ensureIOException(e.getCause());
1420     }
1421   }
1422 
1423   private IOException convertInterruptedExceptionToIOException(final InterruptedException ie) {
1424     Thread.currentThread().interrupt();
1425     IOException ioe = new InterruptedIOException();
1426     ioe.initCause(ie);
1427     return ioe;
1428   }
1429 
1430   private SyncFuture getSyncFuture(final long sequence, Span span) {
1431     SyncFuture syncFuture = this.syncFuturesByHandler.get(Thread.currentThread());
1432     if (syncFuture == null) {
1433       syncFuture = new SyncFuture();
1434       this.syncFuturesByHandler.put(Thread.currentThread(), syncFuture);
1435     }
1436     return syncFuture.reset(sequence, span);
1437   }
1438 
1439   @Override
1440   public void postSync(final long timeInNanos, final int handlerSyncs) {
1441     // TODO: Add metric for handler syncs done at a time.
1442     if (this.metrics != null) metrics.finishSync(timeInNanos/1000000);
1443     if (timeInNanos > this.slowSyncNs) {
1444       String msg =
1445           new StringBuilder().append("Slow sync cost: ")
1446               .append(timeInNanos / 1000000).append(" ms, current pipeline: ")
1447               .append(Arrays.toString(getPipeLine())).toString();
1448       Trace.addTimelineAnnotation(msg);
1449       LOG.info(msg);
1450     }
1451   }
1452 
1453   @Override
1454   public long postAppend(final Entry e, final long elapsedTime) {
1455     long len = 0;
1456     if (this.metrics == null) return len;
1457     for (KeyValue kv : e.getEdit().getKeyValues()) len += kv.getLength();
1458     metrics.finishAppend(elapsedTime, len);
1459     return len;
1460   }
1461 
1462   /**
1463    * This method gets the datanode replication count for the current HLog.
1464    *
1465    * If the pipeline isn't started yet or is empty, you will get the default
1466    * replication factor.  Therefore, if this function returns 0, it means you
1467    * are not properly running with the HDFS-826 patch.
1468    * @throws InvocationTargetException
1469    * @throws IllegalAccessException
1470    * @throws IllegalArgumentException
1471    *
1472    * @throws Exception
1473    */
1474   int getLogReplication()
1475   throws IllegalArgumentException, IllegalAccessException, InvocationTargetException {
1476     if (this.getNumCurrentReplicas != null && this.hdfs_out != null) {
1477       Object repl = this.getNumCurrentReplicas.invoke(getOutputStream(), NO_ARGS);
1478       if (repl instanceof Integer) {
1479         return ((Integer)repl).intValue();
1480       }
1481     }
1482     return 0;
1483   }
1484 
1485   boolean canGetCurReplicas() {
1486     return this.getNumCurrentReplicas != null;
1487   }
1488 
1489   @Override
1490   public void hsync() throws IOException {
1491     TraceScope scope = Trace.startSpan("FSHLog.hsync");
1492     try {
1493       scope = Trace.continueSpan(publishSyncThenBlockOnCompletion(scope.detach()));
1494     } finally {
1495       assert scope == NullScope.INSTANCE || !scope.isDetached();
1496       scope.close();
1497     }
1498   }
1499 
1500   @Override
1501   public void hflush() throws IOException {
1502     TraceScope scope = Trace.startSpan("FSHLog.hflush");
1503     try {
1504       scope = Trace.continueSpan(publishSyncThenBlockOnCompletion(scope.detach()));
1505     } finally {
1506       assert scope == NullScope.INSTANCE || !scope.isDetached();
1507       scope.close();
1508     }
1509   }
1510 
1511   @Override
1512   public void sync() throws IOException {
1513     TraceScope scope = Trace.startSpan("FSHLog.sync");
1514     try {
1515       scope = Trace.continueSpan(publishSyncThenBlockOnCompletion(scope.detach()));
1516     } finally {
1517       assert scope == NullScope.INSTANCE || !scope.isDetached();
1518       scope.close();
1519     }
1520   }
1521 
1522   @Override
1523   public void sync(long txid) throws IOException {
1524     if (this.highestSyncedSequence.get() >= txid){
1525       // Already sync'd.
1526       return;
1527     }
1528     TraceScope scope = Trace.startSpan("FSHLog.sync");
1529     try {
1530       scope = Trace.continueSpan(publishSyncThenBlockOnCompletion(scope.detach()));
1531     } finally {
1532       assert scope == NullScope.INSTANCE || !scope.isDetached();
1533       scope.close();
1534     }
1535   }
1536 
1537   void requestLogRoll() {
1538     if (!this.listeners.isEmpty()) {
1539       for (WALActionsListener i: this.listeners) {
1540         i.logRollRequested();
1541       }
1542     }
1543   }
1544 
1545   /** @return How many items have been added to the log */
1546   int getNumEntries() {
1547     return numEntries.get();
1548   }
1549 
1550   /** @return the number of rolled log files */
1551   public int getNumRolledLogFiles() {
1552     return byWalRegionSequenceIds.size();
1553   }
1554 
1555   /** @return the number of log files in use */
1556   @Override
1557   public int getNumLogFiles() {
1558     // +1 for current use log
1559     return getNumRolledLogFiles() + 1;
1560   }
1561   
1562   /** @return the size of log files in use */
1563   @Override
1564   public long getLogFileSize() {
1565     return this.totalLogSize.get();
1566   }
1567   
1568   @Override
1569   public boolean startCacheFlush(final byte[] encodedRegionName) {
1570     Long oldRegionSeqNum = null;
1571     if (!closeBarrier.beginOp()) {
1572       LOG.info("Flush will not be started for " + Bytes.toString(encodedRegionName) +
1573         " - because the server is closing.");
1574       return false;
1575     }
1576     synchronized (regionSequenceIdLock) {
1577       oldRegionSeqNum = this.oldestUnflushedRegionSequenceIds.remove(encodedRegionName);
1578       if (oldRegionSeqNum != null) {
1579         Long oldValue =
1580           this.lowestFlushingRegionSequenceIds.put(encodedRegionName, oldRegionSeqNum);
1581         assert oldValue ==
1582           null : "Flushing map not cleaned up for " + Bytes.toString(encodedRegionName);
1583       }
1584     }
1585     if (oldRegionSeqNum == null) {
1586       // TODO: if we have no oldRegionSeqNum, and WAL is not disabled, presumably either
1587       //       the region is already flushing (which would make this call invalid), or there
1588       //       were no appends after last flush, so why are we starting flush? Maybe we should
1589       //       assert not null, and switch to "long" everywhere. Less rigorous, but safer,
1590       //       alternative is telling the caller to stop. For now preserve old logic.
1591       LOG.warn("Couldn't find oldest seqNum for the region we are about to flush: ["
1592         + Bytes.toString(encodedRegionName) + "]");
1593     }
1594     return true;
1595   }
1596 
1597   @Override
1598   public void completeCacheFlush(final byte [] encodedRegionName) {
1599     synchronized (regionSequenceIdLock) {
1600       this.lowestFlushingRegionSequenceIds.remove(encodedRegionName);
1601     }
1602     closeBarrier.endOp();
1603   }
1604 
1605   @Override
1606   public void abortCacheFlush(byte[] encodedRegionName) {
1607     Long currentSeqNum = null, seqNumBeforeFlushStarts = null;
1608     synchronized (regionSequenceIdLock) {
1609       seqNumBeforeFlushStarts = this.lowestFlushingRegionSequenceIds.remove(encodedRegionName);
1610       if (seqNumBeforeFlushStarts != null) {
1611         currentSeqNum =
1612           this.oldestUnflushedRegionSequenceIds.put(encodedRegionName, seqNumBeforeFlushStarts);
1613       }
1614     }
1615     closeBarrier.endOp();
1616     if ((currentSeqNum != null)
1617         && (currentSeqNum.longValue() <= seqNumBeforeFlushStarts.longValue())) {
1618       String errorStr = "Region " + Bytes.toString(encodedRegionName) +
1619           "acquired edits out of order current memstore seq=" + currentSeqNum
1620           + ", previous oldest unflushed id=" + seqNumBeforeFlushStarts;
1621       LOG.error(errorStr);
1622       assert false : errorStr;
1623       Runtime.getRuntime().halt(1);
1624     }
1625   }
1626 
1627   @Override
1628   public boolean isLowReplicationRollEnabled() {
1629       return lowReplicationRollEnabled;
1630   }
1631 
1632   /**
1633    * Get the directory we are making logs in.
1634    *
1635    * @return dir
1636    */
1637   protected Path getDir() {
1638     return fullPathLogDir;
1639   }
1640 
1641   static Path getHLogArchivePath(Path oldLogDir, Path p) {
1642     return new Path(oldLogDir, p.getName());
1643   }
1644 
1645   static String formatRecoveredEditsFileName(final long seqid) {
1646     return String.format("%019d", seqid);
1647   }
1648 
1649   public static final long FIXED_OVERHEAD = ClassSize.align(
1650     ClassSize.OBJECT + (5 * ClassSize.REFERENCE) +
1651     ClassSize.ATOMIC_INTEGER + Bytes.SIZEOF_INT + (3 * Bytes.SIZEOF_LONG));
1652 
1653   private static void split(final Configuration conf, final Path p)
1654   throws IOException {
1655     FileSystem fs = FileSystem.get(conf);
1656     if (!fs.exists(p)) {
1657       throw new FileNotFoundException(p.toString());
1658     }
1659     if (!fs.getFileStatus(p).isDirectory()) {
1660       throw new IOException(p + " is not a directory");
1661     }
1662 
1663     final Path baseDir = FSUtils.getRootDir(conf);
1664     final Path oldLogDir = new Path(baseDir, HConstants.HREGION_OLDLOGDIR_NAME);
1665     HLogSplitter.split(baseDir, p, oldLogDir, fs, conf);
1666   }
1667 
1668   @Override
1669   public WALCoprocessorHost getCoprocessorHost() {
1670     return coprocessorHost;
1671   }
1672 
1673   @Override
1674   public long getEarliestMemstoreSeqNum(byte[] encodedRegionName) {
1675     Long result = oldestUnflushedRegionSequenceIds.get(encodedRegionName);
1676     return result == null ? HConstants.NO_SEQNUM : result.longValue();
1677   }
1678 
1679   /**
1680    * This class is used coordinating two threads holding one thread at a
1681    * 'safe point' while the orchestrating thread does some work that requires the first thread
1682    * paused: e.g. holding the WAL writer while its WAL is swapped out from under it by another
1683    * thread.
1684    * 
1685    * <p>Thread A signals Thread B to hold when it gets to a 'safe point'.  Thread A wait until
1686    * Thread B gets there. When the 'safe point' has been attained, Thread B signals Thread A.
1687    * Thread B then holds at the 'safe point'.  Thread A on notification that Thread B is paused,
1688    * goes ahead and does the work it needs to do while Thread B is holding.  When Thread A is done,
1689    * it flags B and then Thread A and Thread B continue along on their merry way.  Pause and
1690    * signalling 'zigzags' between the two participating threads.  We use two latches -- one the
1691    * inverse of the other -- pausing and signaling when states are achieved.
1692    * 
1693    * <p>To start up the drama, Thread A creates an instance of this class each time it would do
1694    * this zigzag dance and passes it to Thread B (these classes use Latches so it is one shot
1695    * only). Thread B notices the new instance (via reading a volatile reference or how ever) and it
1696    * starts to work toward the 'safe point'.  Thread A calls {@link #waitSafePoint()} when it
1697    * cannot proceed until the Thread B 'safe point' is attained. Thread A will be held inside in
1698    * {@link #waitSafePoint()} until Thread B reaches the 'safe point'.  Once there, Thread B
1699    * frees Thread A by calling {@link #safePointAttained()}.  Thread A now knows Thread B
1700    * is at the 'safe point' and that it is holding there (When Thread B calls
1701    * {@link #safePointAttained()} it blocks here until Thread A calls {@link #releaseSafePoint()}).
1702    * Thread A proceeds to do what it needs to do while Thread B is paused.  When finished,
1703    * it lets Thread B lose by calling {@link #releaseSafePoint()} and away go both Threads again.
1704    */
1705   static class SafePointZigZagLatch {
1706     /**
1707      * Count down this latch when safe point attained.
1708      */
1709     private volatile CountDownLatch safePointAttainedLatch = new CountDownLatch(1);
1710     /**
1711      * Latch to wait on.  Will be released when we can proceed.
1712      */
1713     private volatile CountDownLatch safePointReleasedLatch = new CountDownLatch(1);
1714  
1715     /**
1716      * For Thread A to call when it is ready to wait on the 'safe point' to be attained.
1717      * Thread A will be held in here until Thread B calls {@link #safePointAttained()}
1718      * @throws InterruptedException
1719      * @throws ExecutionException
1720      * @param syncFuture We need this as barometer on outstanding syncs.  If it comes home with
1721      * an exception, then something is up w/ our syncing.
1722      * @return The passed <code>syncFuture</code>
1723      * @throws FailedSyncBeforeLogCloseException 
1724      */
1725     SyncFuture waitSafePoint(final SyncFuture syncFuture)
1726     throws InterruptedException, FailedSyncBeforeLogCloseException {
1727       while (true) {
1728         if (this.safePointAttainedLatch.await(1, TimeUnit.NANOSECONDS)) break;
1729         if (syncFuture.isThrowable()) {
1730           throw new FailedSyncBeforeLogCloseException(syncFuture.getThrowable());
1731         }
1732       }
1733       return syncFuture;
1734     }
1735  
1736     /**
1737      * Called by Thread B when it attains the 'safe point'.  In this method, Thread B signals
1738      * Thread A it can proceed. Thread B will be held in here until {@link #releaseSafePoint()}
1739      * is called by Thread A.
1740      * @throws InterruptedException
1741      */
1742     void safePointAttained() throws InterruptedException {
1743       this.safePointAttainedLatch.countDown();
1744       this.safePointReleasedLatch.await();
1745     }
1746 
1747     /**
1748      * Called by Thread A when it is done with the work it needs to do while Thread B is
1749      * halted.  This will release the Thread B held in a call to {@link #safePointAttained()}
1750      */
1751     void releaseSafePoint() {
1752       this.safePointReleasedLatch.countDown();
1753     }
1754 
1755     /**
1756      * @return True is this is a 'cocked', fresh instance, and not one that has already fired.
1757      */
1758     boolean isCocked() {
1759       return this.safePointAttainedLatch.getCount() > 0 &&
1760         this.safePointReleasedLatch.getCount() > 0;
1761     }
1762   }
1763 
1764   /**
1765    * Handler that is run by the disruptor ringbuffer consumer. Consumer is a SINGLE
1766    * 'writer/appender' thread.  Appends edits and starts up sync runs.  Tries its best to batch up
1767    * syncs.  There is no discernible benefit batching appends so we just append as they come in
1768    * because it simplifies the below implementation.  See metrics for batching effectiveness
1769    * (In measurement, at 100 concurrent handlers writing 1k, we are batching > 10 appends and 10
1770    * handler sync invocations for every actual dfsclient sync call; at 10 concurrent handlers,
1771    * YMMV).
1772    * <p>Herein, we have an array into which we store the sync futures as they come in.  When we
1773    * have a 'batch', we'll then pass what we have collected to a SyncRunner thread to do the
1774    * filesystem sync.  When it completes, it will then call
1775    * {@link SyncFuture#done(long, Throwable)} on each of SyncFutures in the batch to release
1776    * blocked Handler threads.
1777    * <p>I've tried various effects to try and make latencies low while keeping throughput high.
1778    * I've tried keeping a single Queue of SyncFutures in this class appending to its tail as the
1779    * syncs coming and having sync runner threads poll off the head to 'finish' completed
1780    * SyncFutures.  I've tried linkedlist, and various from concurrent utils whether
1781    * LinkedBlockingQueue or ArrayBlockingQueue, etc.  The more points of synchronization, the
1782    * more 'work' (according to 'perf stats') that has to be done; small increases in stall
1783    * percentages seem to have a big impact on throughput/latencies.  The below model where we have
1784    * an array into which we stash the syncs and then hand them off to the sync thread seemed like
1785    * a decent compromise.  See HBASE-8755 for more detail.
1786    */
1787   class RingBufferEventHandler implements EventHandler<RingBufferTruck>, LifecycleAware {
1788     private final SyncRunner [] syncRunners;
1789     private final SyncFuture [] syncFutures;
1790     // Had 'interesting' issues when this was non-volatile.  On occasion, we'd not pass all
1791     // syncFutures to the next sync'ing thread.
1792     private volatile int syncFuturesCount = 0;
1793     private volatile SafePointZigZagLatch zigzagLatch;
1794     /**
1795      * Object to block on while waiting on safe point.
1796      */
1797     private final Object safePointWaiter = new Object();
1798     private volatile boolean shutdown = false;
1799 
1800     /**
1801      * Which syncrunner to use next.
1802      */
1803     private int syncRunnerIndex;
1804 
1805     RingBufferEventHandler(final int syncRunnerCount, final int maxHandlersCount) {
1806       this.syncFutures = new SyncFuture[maxHandlersCount];
1807       this.syncRunners = new SyncRunner[syncRunnerCount];
1808       for (int i = 0; i < syncRunnerCount; i++) {
1809         this.syncRunners[i] = new SyncRunner("sync." + i, maxHandlersCount);
1810       }
1811     }
1812 
1813     private void cleanupOutstandingSyncsOnException(final long sequence, final Exception e) {
1814       for (int i = 0; i < this.syncFuturesCount; i++) this.syncFutures[i].done(sequence, e);
1815       this.syncFuturesCount = 0;
1816     }
1817 
1818     @Override
1819     // We can set endOfBatch in the below method if at end of our this.syncFutures array
1820     public void onEvent(final RingBufferTruck truck, final long sequence, boolean endOfBatch)
1821     throws Exception {
1822       // Appends and syncs are coming in order off the ringbuffer.  We depend on this fact.  We'll
1823       // add appends to dfsclient as they come in.  Batching appends doesn't give any significant
1824       // benefit on measurement.  Handler sync calls we will batch up.
1825 
1826       try {
1827         if (truck.hasSyncFuturePayload()) {
1828           this.syncFutures[this.syncFuturesCount++] = truck.unloadSyncFuturePayload();
1829           // Force flush of syncs if we are carrying a full complement of syncFutures.
1830           if (this.syncFuturesCount == this.syncFutures.length) endOfBatch = true;
1831         } else if (truck.hasFSWALEntryPayload()) {
1832           TraceScope scope = Trace.continueSpan(truck.unloadSpanPayload());
1833           try {
1834             append(truck.unloadFSWALEntryPayload());
1835           } catch (Exception e) {
1836             // If append fails, presume any pending syncs will fail too; let all waiting handlers
1837             // know of the exception.
1838             cleanupOutstandingSyncsOnException(sequence, e);
1839             // Return to keep processing.
1840             return;
1841           } finally {
1842             assert scope == NullScope.INSTANCE || !scope.isDetached();
1843             scope.close(); // append scope is complete
1844           }
1845         } else {
1846           // They can't both be null.  Fail all up to this!!!
1847           cleanupOutstandingSyncsOnException(sequence,
1848             new IllegalStateException("Neither append nor sync"));
1849           // Return to keep processing.
1850           return;
1851         }
1852 
1853         // TODO: Check size and if big go ahead and call a sync if we have enough data.
1854 
1855         // If not a batch, return to consume more events from the ring buffer before proceeding;
1856         // we want to get up a batch of syncs and appends before we go do a filesystem sync.
1857         if (!endOfBatch || this.syncFuturesCount <= 0) return;
1858 
1859         // Now we have a batch.
1860 
1861         if (LOG.isTraceEnabled()) {
1862           LOG.trace("Sequence=" + sequence + ", syncCount=" + this.syncFuturesCount);
1863         }
1864 
1865         // Below expects that the offer 'transfers' responsibility for the outstanding syncs to the
1866         // syncRunner.
1867         int index = Math.abs(this.syncRunnerIndex++) % this.syncRunners.length;
1868         this.syncRunners[index].offer(sequence, this.syncFutures, this.syncFuturesCount);
1869         attainSafePoint(sequence);
1870         this.syncFuturesCount = 0;
1871       } catch (Throwable t) {
1872         LOG.error("UNEXPECTED!!! syncFutures.length=" + this.syncFutures.length, t);
1873       }
1874     }
1875 
1876     SafePointZigZagLatch attainSafePoint() {
1877       this.zigzagLatch = new SafePointZigZagLatch();
1878       return this.zigzagLatch;
1879     }
1880 
1881     /**
1882      * Check if we should attain safe point.  If so, go there and then wait till signalled before
1883      * we proceeding.
1884      */
1885     private void attainSafePoint(final long currentSequence) {
1886       if (this.zigzagLatch == null || !this.zigzagLatch.isCocked()) return;
1887       // If here, another thread is waiting on us to get to safe point.  Don't leave it hanging.
1888       try {
1889         // Wait on outstanding syncers; wait for them to finish syncing (unless we've been
1890         // shutdown or unless our latch has been thrown because we have been aborted).
1891         while (!this.shutdown && this.zigzagLatch.isCocked() &&
1892             highestSyncedSequence.get() < currentSequence) {
1893           synchronized (this.safePointWaiter) {
1894             this.safePointWaiter.wait(0, 1);
1895           }
1896         }
1897         // Tell waiting thread we've attained safe point
1898         this.zigzagLatch.safePointAttained();
1899       } catch (InterruptedException e) {
1900         LOG.warn("Interrupted ", e);
1901         Thread.currentThread().interrupt();
1902       }
1903     }
1904 
1905     /**
1906      * Append to the WAL.  Does all CP and WAL listener calls.
1907      * @param entry
1908      * @throws Exception
1909      */
1910     void append(final FSWALEntry entry) throws Exception {
1911       // TODO: WORK ON MAKING THIS APPEND FASTER. DOING WAY TOO MUCH WORK WITH CPs, PBing, etc.
1912       atHeadOfRingBufferEventHandlerAppend();
1913 
1914       long start = EnvironmentEdgeManager.currentTimeMillis();
1915       byte [] encodedRegionName = entry.getKey().getEncodedRegionName();
1916       long regionSequenceId = HLog.NO_SEQUENCE_ID;
1917       try {
1918         // We are about to append this edit; update the region-scoped sequence number.  Do it
1919         // here inside this single appending/writing thread.  Events are ordered on the ringbuffer
1920         // so region sequenceids will also be in order.
1921         regionSequenceId = entry.stampRegionSequenceId();
1922         
1923         // Edits are empty, there is nothing to append.  Maybe empty when we are looking for a 
1924         // region sequence id only, a region edit/sequence id that is not associated with an actual 
1925         // edit. It has to go through all the rigmarole to be sure we have the right ordering.
1926         if (entry.getEdit().isEmpty()) {
1927           return;
1928         }
1929         
1930         // Coprocessor hook.
1931         if (!coprocessorHost.preWALWrite(entry.getHRegionInfo(), entry.getKey(),
1932             entry.getEdit())) {
1933           if (entry.getEdit().isReplay()) {
1934             // Set replication scope null so that this won't be replicated
1935             entry.getKey().setScopes(null);
1936           }
1937         }
1938         if (!listeners.isEmpty()) {
1939           for (WALActionsListener i: listeners) {
1940             // TODO: Why does listener take a table description and CPs take a regioninfo?  Fix.
1941             i.visitLogEntryBeforeWrite(entry.getHTableDescriptor(), entry.getKey(),
1942               entry.getEdit());
1943           }
1944         }
1945 
1946         writer.append(entry);
1947         assert highestUnsyncedSequence < entry.getSequence();
1948         highestUnsyncedSequence = entry.getSequence();
1949         Long lRegionSequenceId = Long.valueOf(regionSequenceId);
1950         highestRegionSequenceIds.put(encodedRegionName, lRegionSequenceId);
1951         if (entry.isInMemstore()) {
1952           oldestUnflushedRegionSequenceIds.putIfAbsent(encodedRegionName, lRegionSequenceId);
1953         }
1954         
1955         coprocessorHost.postWALWrite(entry.getHRegionInfo(), entry.getKey(), entry.getEdit());
1956         // Update metrics.
1957         postAppend(entry, EnvironmentEdgeManager.currentTimeMillis() - start);
1958       } catch (Exception e) {
1959         LOG.fatal("Could not append. Requesting close of hlog", e);
1960         requestLogRoll();
1961         throw e;
1962       }
1963       numEntries.incrementAndGet();
1964     }
1965 
1966     @Override
1967     public void onStart() {
1968       for (SyncRunner syncRunner: this.syncRunners) syncRunner.start();
1969     }
1970 
1971     @Override
1972     public void onShutdown() {
1973       for (SyncRunner syncRunner: this.syncRunners) syncRunner.interrupt();
1974     }
1975   }
1976 
1977   /**
1978    * Exposed for testing only.  Use to tricks like halt the ring buffer appending.
1979    */
1980   @VisibleForTesting
1981   void atHeadOfRingBufferEventHandlerAppend() {
1982     // Noop
1983   }
1984 
1985   private static IOException ensureIOException(final Throwable t) {
1986     return (t instanceof IOException)? (IOException)t: new IOException(t);
1987   }
1988 
1989   private static void usage() {
1990     System.err.println("Usage: HLog <ARGS>");
1991     System.err.println("Arguments:");
1992     System.err.println(" --dump  Dump textual representation of passed one or more files");
1993     System.err.println("         For example: " +
1994       "FSHLog --dump hdfs://example.com:9000/hbase/.logs/MACHINE/LOGFILE");
1995     System.err.println(" --split Split the passed directory of WAL logs");
1996     System.err.println("         For example: " +
1997       "FSHLog --split hdfs://example.com:9000/hbase/.logs/DIR");
1998     System.err.println(" --perf  Write the same key <N> times to a WAL: e.g. FSHLog --perf 10");
1999   }
2000 
2001   /**
2002    * Pass one or more log file names and it will either dump out a text version
2003    * on <code>stdout</code> or split the specified log files.
2004    *
2005    * @param args
2006    * @throws IOException
2007    */
2008   public static void main(String[] args) throws IOException {
2009     if (args.length < 2) {
2010       usage();
2011       System.exit(-1);
2012     }
2013     // either dump using the HLogPrettyPrinter or split, depending on args
2014     if (args[0].compareTo("--dump") == 0) {
2015       HLogPrettyPrinter.run(Arrays.copyOfRange(args, 1, args.length));
2016     } else if (args[0].compareTo("--perf") == 0) {
2017       final int count = Integer.parseInt(args[1]);
2018       // Put up a WAL and just keep adding same edit to it.  Simple perf test.
2019       Configuration conf = HBaseConfiguration.create();
2020       Path rootDir = FSUtils.getRootDir(conf);
2021       FileSystem fs = rootDir.getFileSystem(conf);
2022       FSHLog wal =
2023         new FSHLog(fs, rootDir, "perflog", "oldPerflog", conf, null, false, "perf", false);
2024       long start = System.nanoTime();
2025       WALEdit walEdit = new WALEdit();
2026       walEdit.add(new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("family"),
2027         Bytes.toBytes("qualifier"), -1, new byte [1000]));
2028       for (AtomicLong i = new AtomicLong(0); i.get() < count; i.incrementAndGet()) {
2029         wal.append(HRegionInfo.FIRST_META_REGIONINFO, TableName.META_TABLE_NAME, walEdit, start,
2030           HTableDescriptor.META_TABLEDESC, i);
2031         wal.sync();
2032       }
2033       wal.close();
2034       LOG.info("Write " + count + " 1k edits in " + (System.nanoTime() - start) + "nanos");
2035     } else if (args[0].compareTo("--split") == 0) {
2036       Configuration conf = HBaseConfiguration.create();
2037       for (int i = 1; i < args.length; i++) {
2038         try {
2039           Path logPath = new Path(args[i]);
2040           FSUtils.setFsDefault(conf, logPath);
2041           split(conf, logPath);
2042         } catch (IOException t) {
2043           t.printStackTrace(System.err);
2044           System.exit(-1);
2045         }
2046       }
2047     } else {
2048       usage();
2049       System.exit(-1);
2050     }
2051   }
2052   
2053   /**
2054    * Find the 'getPipeline' on the passed <code>os</code> stream.
2055    * @return Method or null.
2056    */
2057   private Method getGetPipeline(final FSDataOutputStream os) {
2058     Method m = null;
2059     if (os != null) {
2060       Class<? extends OutputStream> wrappedStreamClass = os.getWrappedStream()
2061           .getClass();
2062       try {
2063         m = wrappedStreamClass.getDeclaredMethod("getPipeline",
2064           new Class<?>[] {});
2065         m.setAccessible(true);
2066       } catch (NoSuchMethodException e) {
2067         LOG.info("FileSystem's output stream doesn't support"
2068             + " getPipeline; not available; fsOut="
2069             + wrappedStreamClass.getName());
2070       } catch (SecurityException e) {
2071         LOG.info(
2072           "Doesn't have access to getPipeline on "
2073               + "FileSystems's output stream ; fsOut="
2074               + wrappedStreamClass.getName(), e);
2075         m = null; // could happen on setAccessible()
2076       }
2077     }
2078     return m;
2079   }
2080 
2081   /**
2082    * This method gets the pipeline for the current HLog.
2083    * @return
2084    */
2085   DatanodeInfo[] getPipeLine() {
2086     if (this.getPipeLine != null && this.hdfs_out != null) {
2087       Object repl;
2088       try {
2089         repl = this.getPipeLine.invoke(getOutputStream(), NO_ARGS);
2090         if (repl instanceof DatanodeInfo[]) {
2091           return ((DatanodeInfo[]) repl);
2092         }
2093       } catch (Exception e) {
2094         LOG.info("Get pipeline failed", e);
2095       }
2096     }
2097     return new DatanodeInfo[0];
2098   }
2099 }