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    * In the case of creating a new WAL, oldPath will be null.
829    *
830    * In the case of rolling over from one file to the next, none of the params will be null.
831    *
832    * In the case of closing out this FSHLog with no further use newPath, nextWriter, and
833    * nextHdfsOut will be null.
834    *
835    * @param oldPath may be null
836    * @param newPath may be null
837    * @param nextWriter may be null
838    * @param nextHdfsOut may be null
839    * @return the passed in <code>newPath</code>
840    * @throws IOException if there is a problem flushing or closing the underlying FS
841    */
842   Path replaceWriter(final Path oldPath, final Path newPath, FSHLog.Writer nextWriter,
843       final FSDataOutputStream nextHdfsOut)
844   throws IOException {
845     // Ask the ring buffer writer to pause at a safe point.  Once we do this, the writer
846     // thread will eventually pause. An error hereafter needs to release the writer thread
847     // regardless -- hence the finally block below.  Note, this method is called from the FSHLog
848     // constructor BEFORE the ring buffer is set running so it is null on first time through
849     // here; allow for that.
850     SyncFuture syncFuture = null;
851     SafePointZigZagLatch zigzagLatch = (this.ringBufferEventHandler == null)?
852       null: this.ringBufferEventHandler.attainSafePoint();
853     TraceScope scope = Trace.startSpan("FSHFile.replaceWriter");
854     try {
855       // Wait on the safe point to be achieved.  Send in a sync in case nothing has hit the
856       // ring buffer between the above notification of writer that we want it to go to
857       // 'safe point' and then here where we are waiting on it to attain safe point.  Use
858       // 'sendSync' instead of 'sync' because we do not want this thread to block waiting on it
859       // to come back.  Cleanup this syncFuture down below after we are ready to run again.
860       try {
861         if (zigzagLatch != null) {
862           Trace.addTimelineAnnotation("awaiting safepoint");
863           syncFuture = zigzagLatch.waitSafePoint(publishSyncOnRingBuffer());
864         }
865       } catch (FailedSyncBeforeLogCloseException e) {
866         if (isUnflushedEntries()) throw e;
867         // Else, let is pass through to the close.
868         LOG.warn("Failed last sync but no outstanding unsync edits so falling through to close; " +
869           e.getMessage());
870       }
871 
872       // It is at the safe point.  Swap out writer from under the blocked writer thread.
873       // TODO: This is close is inline with critical section.  Should happen in background?
874       try {
875         if (this.writer != null) {
876           Trace.addTimelineAnnotation("closing writer");
877           this.writer.close();
878           Trace.addTimelineAnnotation("writer closed");
879         }
880         this.closeErrorCount.set(0);
881       } catch (IOException ioe) {
882         int errors = closeErrorCount.incrementAndGet();
883         if (!isUnflushedEntries() && (errors <= this.closeErrorsTolerated)) {
884           LOG.warn("Riding over failed WAL close of " + oldPath + ", cause=\"" +
885             ioe.getMessage() + "\", errors=" + errors +
886             "; THIS FILE WAS NOT CLOSED BUT ALL EDITS SYNCED SO SHOULD BE OK");
887         } else {
888           throw ioe;
889         }
890       }
891       this.writer = nextWriter;
892       this.hdfs_out = nextHdfsOut;
893       int oldNumEntries = this.numEntries.get();
894       this.numEntries.set(0);
895       final String newPathString = (null == newPath ? null : FSUtils.getPath(newPath));
896       if (oldPath != null) {
897         this.byWalRegionSequenceIds.put(oldPath, this.highestRegionSequenceIds);
898         this.highestRegionSequenceIds = new HashMap<byte[], Long>();
899         long oldFileLen = this.fs.getFileStatus(oldPath).getLen();
900         this.totalLogSize.addAndGet(oldFileLen);
901         LOG.info("Rolled WAL " + FSUtils.getPath(oldPath) + " with entries=" + oldNumEntries +
902           ", filesize=" + StringUtils.byteDesc(oldFileLen) + "; new WAL " +
903           newPathString);
904       } else {
905         LOG.info("New WAL " + newPathString);
906       }
907     } catch (InterruptedException ie) {
908       // Perpetuate the interrupt
909       Thread.currentThread().interrupt();
910     } catch (IOException e) {
911       long count = getUnflushedEntriesCount();
912       LOG.error("Failed close of HLog writer " + oldPath + ", unflushedEntries=" + count, e);
913       throw new FailedLogCloseException(oldPath + ", unflushedEntries=" + count, e);
914     } finally {
915       try {
916         // Let the writer thread go regardless, whether error or not.
917         if (zigzagLatch != null) {
918           zigzagLatch.releaseSafePoint();
919           // It will be null if we failed our wait on safe point above.
920           if (syncFuture != null) blockOnSync(syncFuture);
921         }
922       } finally {
923         scope.close();
924       }
925     }
926     return newPath;
927   }
928 
929   long getUnflushedEntriesCount() {
930     long highestSynced = this.highestSyncedSequence.get();
931     return highestSynced > this.highestUnsyncedSequence?
932       0: this.highestUnsyncedSequence - highestSynced;
933   }
934 
935   boolean isUnflushedEntries() {
936     return getUnflushedEntriesCount() > 0;
937   }
938 
939   private void archiveLogFile(final Path p) throws IOException {
940     Path newPath = getHLogArchivePath(this.fullPathOldLogDir, p);
941     // Tell our listeners that a log is going to be archived.
942     if (!this.listeners.isEmpty()) {
943       for (WALActionsListener i : this.listeners) {
944         i.preLogArchive(p, newPath);
945       }
946     }
947     if (!FSUtils.renameAndSetModifyTime(this.fs, p, newPath)) {
948       throw new IOException("Unable to rename " + p + " to " + newPath);
949     }
950     // Tell our listeners that a log has been archived.
951     if (!this.listeners.isEmpty()) {
952       for (WALActionsListener i : this.listeners) {
953         i.postLogArchive(p, newPath);
954       }
955     }
956   }
957 
958   /**
959    * This is a convenience method that computes a new filename with a given
960    * file-number.
961    * @param filenum to use
962    * @return Path
963    */
964   protected Path computeFilename(final long filenum) {
965     this.filenum.set(filenum);
966     return computeFilename();
967   }
968 
969   /**
970    * This is a convenience method that computes a new filename with a given
971    * using the current HLog file-number
972    * @return Path
973    */
974   protected Path computeFilename() {
975     if (this.filenum.get() < 0) {
976       throw new RuntimeException("hlog file number can't be < 0");
977     }
978     String child = logFilePrefix + "." + filenum;
979     if (forMeta) {
980       child += HLog.META_HLOG_FILE_EXTN;
981     }
982     return new Path(fullPathLogDir, child);
983   }
984 
985 /**
986  * A log file has a creation timestamp (in ms) in its file name ({@link #filenum}.
987  * This helper method returns the creation timestamp from a given log file.
988  * It extracts the timestamp assuming the filename is created with the
989  * {@link #computeFilename(long filenum)} method.
990  * @param fileName
991  * @return timestamp, as in the log file name.
992  */
993   protected long getFileNumFromFileName(Path fileName) {
994     if (fileName == null) throw new IllegalArgumentException("file name can't be null");
995     // The path should start with dir/<prefix>.
996     String prefixPathStr = new Path(fullPathLogDir, logFilePrefix + ".").toString();
997     if (!fileName.toString().startsWith(prefixPathStr)) {
998       throw new IllegalArgumentException("The log file " + fileName + " doesn't belong to" +
999       		" this regionserver " + prefixPathStr);
1000     }
1001     String chompedPath = fileName.toString().substring(prefixPathStr.length());
1002     if (forMeta) chompedPath = chompedPath.substring(0, chompedPath.indexOf(META_HLOG_FILE_EXTN));
1003     return Long.parseLong(chompedPath);
1004   }
1005 
1006   @Override
1007   public void closeAndDelete() throws IOException {
1008     close();
1009     if (!fs.exists(this.fullPathLogDir)) return;
1010     FileStatus[] files = fs.listStatus(this.fullPathLogDir);
1011     if (files != null) {
1012       for(FileStatus file : files) {
1013 
1014         Path p = getHLogArchivePath(this.fullPathOldLogDir, file.getPath());
1015         // Tell our listeners that a log is going to be archived.
1016         if (!this.listeners.isEmpty()) {
1017           for (WALActionsListener i : this.listeners) {
1018             i.preLogArchive(file.getPath(), p);
1019           }
1020         }
1021 
1022         if (!FSUtils.renameAndSetModifyTime(fs, file.getPath(), p)) {
1023           throw new IOException("Unable to rename " + file.getPath() + " to " + p);
1024         }
1025         // Tell our listeners that a log was archived.
1026         if (!this.listeners.isEmpty()) {
1027           for (WALActionsListener i : this.listeners) {
1028             i.postLogArchive(file.getPath(), p);
1029           }
1030         }
1031       }
1032       LOG.debug("Moved " + files.length + " WAL file(s) to " +
1033         FSUtils.getPath(this.fullPathOldLogDir));
1034     }
1035     if (!fs.delete(fullPathLogDir, true)) {
1036       LOG.info("Unable to delete " + fullPathLogDir);
1037     }
1038   }
1039 
1040   @Override
1041   public void close() throws IOException {
1042     if (this.closed) return;
1043     try {
1044       // Prevent all further flushing and rolling.
1045       closeBarrier.stopAndDrainOps();
1046     } catch (InterruptedException e) {
1047       LOG.error("Exception while waiting for cache flushes and log rolls", e);
1048       Thread.currentThread().interrupt();
1049     }
1050 
1051     // Shutdown the disruptor.  Will stop after all entries have been processed.  Make sure we have
1052     // stopped incoming appends before calling this else it will not shutdown.  We are
1053     // conservative below waiting a long time and if not elapsed, then halting.
1054     if (this.disruptor != null) {
1055       long timeoutms = conf.getLong("hbase.wal.disruptor.shutdown.timeout.ms", 60000);
1056       try {
1057         this.disruptor.shutdown(timeoutms, TimeUnit.MILLISECONDS);
1058       } catch (TimeoutException e) {
1059         LOG.warn("Timed out bringing down disruptor after " + timeoutms + "ms; forcing halt " +
1060           "(It is a problem if this is NOT an ABORT! -- DATALOSS!!!!)");
1061         this.disruptor.halt();
1062         this.disruptor.shutdown();
1063       }
1064     }
1065     // With disruptor down, this is safe to let go.
1066     if (this.appendExecutor !=  null) this.appendExecutor.shutdown();
1067 
1068     // Tell our listeners that the log is closing
1069     if (!this.listeners.isEmpty()) {
1070       for (WALActionsListener i : this.listeners) {
1071         i.logCloseRequested();
1072       }
1073     }
1074     this.closed = true;
1075     if (LOG.isDebugEnabled()) {
1076       LOG.debug("Closing WAL writer in " + this.fullPathLogDir.toString());
1077     }
1078     if (this.writer != null) {
1079       this.writer.close();
1080       this.writer = null;
1081     }
1082   }
1083 
1084   /**
1085    * @param now
1086    * @param encodedRegionName Encoded name of the region as returned by
1087    * <code>HRegionInfo#getEncodedNameAsBytes()</code>.
1088    * @param tableName
1089    * @param clusterIds that have consumed the change
1090    * @return New log key.
1091    */
1092   protected HLogKey makeKey(byte[] encodedRegionName, TableName tableName, long seqnum,
1093       long now, List<UUID> clusterIds, long nonceGroup, long nonce) {
1094     return new HLogKey(encodedRegionName, tableName, seqnum, now, clusterIds, nonceGroup, nonce);
1095   }
1096   
1097   @Override
1098   @VisibleForTesting
1099   public void append(HRegionInfo info, TableName tableName, WALEdit edits,
1100     final long now, HTableDescriptor htd, AtomicLong sequenceId)
1101   throws IOException {
1102     HLogKey logKey = new HLogKey(info.getEncodedNameAsBytes(), tableName, now);
1103     append(htd, info, logKey, edits, sequenceId, true, true, null);
1104   }
1105 
1106   @Override
1107   public long appendNoSync(final HRegionInfo info, TableName tableName, WALEdit edits,
1108       List<UUID> clusterIds, final long now, HTableDescriptor htd, AtomicLong sequenceId,
1109       boolean inMemstore, long nonceGroup, long nonce) throws IOException {
1110     HLogKey logKey =
1111       new HLogKey(info.getEncodedNameAsBytes(), tableName, now, clusterIds, nonceGroup, nonce);
1112     return append(htd, info, logKey, edits, sequenceId, false, inMemstore, null);
1113   }
1114 
1115   @Override
1116   public long appendNoSync(final HTableDescriptor htd, final HRegionInfo info, final HLogKey key,
1117       final WALEdit edits, final AtomicLong sequenceId, final boolean inMemstore, 
1118       final List<KeyValue> memstoreKVs)
1119   throws IOException {
1120     return append(htd, info, key, edits, sequenceId, false, inMemstore, memstoreKVs);
1121   }
1122 
1123   /**
1124    * Append a set of edits to the log. Log edits are keyed by (encoded) regionName, rowname, and
1125    * log-sequence-id.
1126    * @param key
1127    * @param edits
1128    * @param htd This comes in here just so it is available on a pre append for replications.  Get
1129    * rid of it.  It is kinda crazy this comes in here when we have tablename and regioninfo.
1130    * Replication gets its scope from the HTD.
1131    * @param hri region info
1132    * @param sync shall we sync after we call the append?
1133    * @param inMemstore
1134    * @param sequenceId The region sequence id reference.
1135    * @param memstoreKVs
1136    * @return txid of this transaction or if nothing to do, the last txid
1137    * @throws IOException
1138    */
1139   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NP_NULL_ON_SOME_PATH_EXCEPTION",
1140       justification="Will never be null")
1141   private long append(HTableDescriptor htd, final HRegionInfo hri, final HLogKey key,
1142       WALEdit edits, AtomicLong sequenceId, boolean sync, boolean inMemstore, 
1143       List<KeyValue> memstoreKVs)
1144   throws IOException {
1145     if (!this.enabled) return this.highestUnsyncedSequence;
1146     if (this.closed) throw new IOException("Cannot append; log is closed");
1147     // Make a trace scope for the append.  It is closed on other side of the ring buffer by the
1148     // single consuming thread.  Don't have to worry about it.
1149     TraceScope scope = Trace.startSpan("FSHLog.append");
1150 
1151     // This is crazy how much it takes to make an edit.  Do we need all this stuff!!!!????  We need
1152     // all this to make a key and then below to append the edit, we need to carry htd, info,
1153     // etc. all over the ring buffer.
1154     FSWALEntry entry = null;
1155     long sequence = this.disruptor.getRingBuffer().next();
1156     try {
1157       RingBufferTruck truck = this.disruptor.getRingBuffer().get(sequence);
1158       // Construction of FSWALEntry sets a latch.  The latch is thrown just after we stamp the
1159       // edit with its edit/sequence id.  The below entry.getRegionSequenceId will wait on the
1160       // latch to be thrown.  TODO: reuse FSWALEntry as we do SyncFuture rather create per append.
1161       entry = new FSWALEntry(sequence, key, edits, sequenceId, inMemstore, htd, hri, memstoreKVs);
1162       truck.loadPayload(entry, scope.detach());
1163     } finally {
1164       this.disruptor.getRingBuffer().publish(sequence);
1165     }
1166     // doSync is set in tests.  Usually we arrive in here via appendNoSync w/ the sync called after
1167     // all edits on a handler have been added.
1168     if (sync) sync(sequence);
1169     return sequence;
1170   }
1171 
1172   /**
1173    * Thread to runs the hdfs sync call. This call takes a while to complete.  This is the longest
1174    * pole adding edits to the WAL and this must complete to be sure all edits persisted.  We run
1175    * multiple threads sync'ng rather than one that just syncs in series so we have better
1176    * latencies; otherwise, an edit that arrived just after a sync started, might have to wait
1177    * almost the length of two sync invocations before it is marked done.
1178    * <p>When the sync completes, it marks all the passed in futures done.  On the other end of the
1179    * sync future is a blocked thread, usually a regionserver Handler.  There may be more than one
1180    * future passed in the case where a few threads arrive at about the same time and all invoke
1181    * 'sync'.  In this case we'll batch up the invocations and run one filesystem sync only for a
1182    * batch of Handler sync invocations.  Do not confuse these Handler SyncFutures with the futures
1183    * an ExecutorService returns when you call submit. We have no use for these in this model. These
1184    * SyncFutures are 'artificial', something to hold the Handler until the filesystem sync
1185    * completes.
1186    */
1187   private class SyncRunner extends HasThread {
1188     private volatile long sequence;
1189     private final BlockingQueue<SyncFuture> syncFutures;
1190  
1191     /**
1192      * UPDATE! 
1193      * @param syncs the batch of calls to sync that arrived as this thread was starting; when done,
1194      * we will put the result of the actual hdfs sync call as the result.
1195      * @param sequence The sequence number on the ring buffer when this thread was set running.
1196      * If this actual writer sync completes then all appends up this point have been
1197      * flushed/synced/pushed to datanodes.  If we fail, then the passed in <code>syncs</code>
1198      * futures will return the exception to their clients; some of the edits may have made it out
1199      * to data nodes but we will report all that were part of this session as failed.
1200      */
1201     SyncRunner(final String name, final int maxHandlersCount) {
1202       super(name);
1203       // LinkedBlockingQueue because of
1204       // http://www.javacodegeeks.com/2010/09/java-best-practices-queue-battle-and.html
1205       // Could use other blockingqueues here or concurrent queues.
1206       this.syncFutures = new LinkedBlockingQueue<SyncFuture>(maxHandlersCount);
1207     }
1208 
1209     void offer(final long sequence, final SyncFuture [] syncFutures, final int syncFutureCount) {
1210       // Set sequence first because the add to the queue will wake the thread if sleeping.
1211       this.sequence = sequence;
1212       for (int i = 0; i < syncFutureCount; i++) this.syncFutures.add(syncFutures[i]);
1213     }
1214 
1215     /**
1216      * Release the passed <code>syncFuture</code>
1217      * @param syncFuture
1218      * @param currentSequence
1219      * @param t
1220      * @return Returns 1.
1221      */
1222     private int releaseSyncFuture(final SyncFuture syncFuture, final long currentSequence,
1223         final Throwable t) {
1224       if (!syncFuture.done(currentSequence, t)) throw new IllegalStateException();
1225       // This function releases one sync future only.
1226       return 1;
1227     }
1228  
1229     /**
1230      * Release all SyncFutures whose sequence is <= <code>currentSequence</code>.
1231      * @param currentSequence
1232      * @param t May be non-null if we are processing SyncFutures because an exception was thrown.
1233      * @return Count of SyncFutures we let go.
1234      */
1235     private int releaseSyncFutures(final long currentSequence, final Throwable t) {
1236       int syncCount = 0;
1237       for (SyncFuture syncFuture; (syncFuture = this.syncFutures.peek()) != null;) {
1238         if (syncFuture.getRingBufferSequence() > currentSequence) break;
1239         releaseSyncFuture(syncFuture, currentSequence, t);
1240         if (!this.syncFutures.remove(syncFuture)) {
1241           throw new IllegalStateException(syncFuture.toString());
1242         }
1243         syncCount++;
1244       }
1245       return syncCount;
1246     }
1247 
1248     /**
1249      * @param sequence The sequence we ran the filesystem sync against.
1250      * @return Current highest synced sequence.
1251      */
1252     private long updateHighestSyncedSequence(long sequence) {
1253       long currentHighestSyncedSequence;
1254       // Set the highestSyncedSequence IFF our current sequence id is the 'highest'.
1255       do {
1256         currentHighestSyncedSequence = highestSyncedSequence.get();
1257         if (currentHighestSyncedSequence >= sequence) {
1258           // Set the sync number to current highwater mark; might be able to let go more
1259           // queued sync futures
1260           sequence = currentHighestSyncedSequence;
1261           break;
1262         }
1263       } while (!highestSyncedSequence.compareAndSet(currentHighestSyncedSequence, sequence));
1264       return sequence;
1265     }
1266 
1267     public void run() {
1268       long currentSequence;
1269       while (!isInterrupted()) {
1270         int syncCount = 0;
1271         SyncFuture takeSyncFuture;
1272         try {
1273           while (true) {
1274             // We have to process what we 'take' from the queue
1275             takeSyncFuture = this.syncFutures.take();
1276             currentSequence = this.sequence;
1277             long syncFutureSequence = takeSyncFuture.getRingBufferSequence();
1278             if (syncFutureSequence > currentSequence) {
1279               throw new IllegalStateException("currentSequence=" + syncFutureSequence +
1280                 ", syncFutureSequence=" + syncFutureSequence);
1281             }
1282             // See if we can process any syncfutures BEFORE we go sync.
1283             long currentHighestSyncedSequence = highestSyncedSequence.get();
1284             if (currentSequence < currentHighestSyncedSequence) {
1285               syncCount += releaseSyncFuture(takeSyncFuture, currentHighestSyncedSequence, null);
1286               // Done with the 'take'.  Go around again and do a new 'take'.
1287               continue;
1288             }
1289             break;
1290           }
1291           // I got something.  Lets run.  Save off current sequence number in case it changes
1292           // while we run.
1293           TraceScope scope = Trace.continueSpan(takeSyncFuture.getSpan());
1294           long start = System.nanoTime();
1295           Throwable t = null;
1296           try {
1297             Trace.addTimelineAnnotation("syncing writer");
1298             writer.sync();
1299             Trace.addTimelineAnnotation("writer synced");
1300             currentSequence = updateHighestSyncedSequence(currentSequence);
1301           } catch (IOException e) {
1302             LOG.error("Error syncing, request close of hlog ", e);
1303             t = e;
1304           } catch (Exception e) {
1305             LOG.warn("UNEXPECTED", e);
1306             t = e;
1307           } finally {
1308             // reattach the span to the future before releasing.
1309             takeSyncFuture.setSpan(scope.detach());
1310             // First release what we 'took' from the queue.
1311             syncCount += releaseSyncFuture(takeSyncFuture, currentSequence, t);
1312             // Can we release other syncs?
1313             syncCount += releaseSyncFutures(currentSequence, t);
1314             if (t != null) {
1315               requestLogRoll();
1316             } else checkLogRoll();
1317           }
1318           postSync(System.nanoTime() - start, syncCount);
1319         } catch (InterruptedException e) {
1320           // Presume legit interrupt.
1321           Thread.currentThread().interrupt();
1322         } catch (Throwable t) {
1323           LOG.warn("UNEXPECTED, continuing", t);
1324         }
1325       }
1326     }
1327   }
1328 
1329   /**
1330    * Schedule a log roll if needed.
1331    */
1332   void checkLogRoll() {
1333     // Will return immediately if we are in the middle of a WAL log roll currently.
1334     if (!rollWriterLock.tryLock()) return;
1335     boolean lowReplication;
1336     try {
1337       lowReplication = checkLowReplication();
1338     } finally {
1339       rollWriterLock.unlock();
1340     }
1341     try {
1342       if (lowReplication || writer != null && writer.getLength() > logrollsize) requestLogRoll();
1343     } catch (IOException e) {
1344       LOG.warn("Writer.getLength() failed; continuing", e);
1345     }
1346   }
1347 
1348   /*
1349    * @return true if number of replicas for the WAL is lower than threshold
1350    */
1351   private boolean checkLowReplication() {
1352     boolean logRollNeeded = false;
1353     // if the number of replicas in HDFS has fallen below the configured
1354     // value, then roll logs.
1355     try {
1356       int numCurrentReplicas = getLogReplication();
1357       if (numCurrentReplicas != 0 && numCurrentReplicas < this.minTolerableReplication) {
1358         if (this.lowReplicationRollEnabled) {
1359           if (this.consecutiveLogRolls.get() < this.lowReplicationRollLimit) {
1360             LOG.warn("HDFS pipeline error detected. " + "Found "
1361                 + numCurrentReplicas + " replicas but expecting no less than "
1362                 + this.minTolerableReplication + " replicas. "
1363                 + " Requesting close of hlog.");
1364             logRollNeeded = true;
1365             // If rollWriter is requested, increase consecutiveLogRolls. Once it
1366             // is larger than lowReplicationRollLimit, disable the
1367             // LowReplication-Roller
1368             this.consecutiveLogRolls.getAndIncrement();
1369           } else {
1370             LOG.warn("Too many consecutive RollWriter requests, it's a sign of "
1371                 + "the total number of live datanodes is lower than the tolerable replicas.");
1372             this.consecutiveLogRolls.set(0);
1373             this.lowReplicationRollEnabled = false;
1374           }
1375         }
1376       } else if (numCurrentReplicas >= this.minTolerableReplication) {
1377         if (!this.lowReplicationRollEnabled) {
1378           // The new writer's log replicas is always the default value.
1379           // So we should not enable LowReplication-Roller. If numEntries
1380           // is lower than or equals 1, we consider it as a new writer.
1381           if (this.numEntries.get() <= 1) {
1382             return logRollNeeded;
1383           }
1384           // Once the live datanode number and the replicas return to normal,
1385           // enable the LowReplication-Roller.
1386           this.lowReplicationRollEnabled = true;
1387           LOG.info("LowReplication-Roller was enabled.");
1388         }
1389       }
1390     } catch (Exception e) {
1391       LOG.warn("Unable to invoke DFSOutputStream.getNumCurrentReplicas" + e +
1392         " still proceeding ahead...");
1393     }
1394     return logRollNeeded;
1395   }
1396 
1397   private SyncFuture publishSyncOnRingBuffer() {
1398     return publishSyncOnRingBuffer(null);
1399   }
1400 
1401   private SyncFuture publishSyncOnRingBuffer(Span span) {
1402     long sequence = this.disruptor.getRingBuffer().next();
1403     SyncFuture syncFuture = getSyncFuture(sequence, span);
1404     try {
1405       RingBufferTruck truck = this.disruptor.getRingBuffer().get(sequence);
1406       truck.loadPayload(syncFuture);
1407     } finally {
1408       this.disruptor.getRingBuffer().publish(sequence);
1409     }
1410     return syncFuture;
1411   }
1412 
1413   // Sync all known transactions
1414   private Span publishSyncThenBlockOnCompletion(Span span) throws IOException {
1415     return blockOnSync(publishSyncOnRingBuffer(span));
1416   }
1417 
1418   private Span blockOnSync(final SyncFuture syncFuture) throws IOException {
1419     // Now we have published the ringbuffer, halt the current thread until we get an answer back.
1420     try {
1421       syncFuture.get();
1422       return syncFuture.getSpan();
1423     } catch (InterruptedException ie) {
1424       LOG.warn("Interrupted", ie);
1425       throw convertInterruptedExceptionToIOException(ie);
1426     } catch (ExecutionException e) {
1427       throw ensureIOException(e.getCause());
1428     }
1429   }
1430 
1431   private IOException convertInterruptedExceptionToIOException(final InterruptedException ie) {
1432     Thread.currentThread().interrupt();
1433     IOException ioe = new InterruptedIOException();
1434     ioe.initCause(ie);
1435     return ioe;
1436   }
1437 
1438   private SyncFuture getSyncFuture(final long sequence, Span span) {
1439     SyncFuture syncFuture = this.syncFuturesByHandler.get(Thread.currentThread());
1440     if (syncFuture == null) {
1441       syncFuture = new SyncFuture();
1442       this.syncFuturesByHandler.put(Thread.currentThread(), syncFuture);
1443     }
1444     return syncFuture.reset(sequence, span);
1445   }
1446 
1447   @Override
1448   public void postSync(final long timeInNanos, final int handlerSyncs) {
1449     // TODO: Add metric for handler syncs done at a time.
1450     if (this.metrics != null) metrics.finishSync(timeInNanos/1000000);
1451     if (timeInNanos > this.slowSyncNs) {
1452       String msg =
1453           new StringBuilder().append("Slow sync cost: ")
1454               .append(timeInNanos / 1000000).append(" ms, current pipeline: ")
1455               .append(Arrays.toString(getPipeLine())).toString();
1456       Trace.addTimelineAnnotation(msg);
1457       LOG.info(msg);
1458     }
1459   }
1460 
1461   @Override
1462   public long postAppend(final Entry e, final long elapsedTime) {
1463     long len = 0;
1464     if (this.metrics == null) return len;
1465     for (KeyValue kv : e.getEdit().getKeyValues()) len += kv.getLength();
1466     metrics.finishAppend(elapsedTime, len);
1467     return len;
1468   }
1469 
1470   /**
1471    * This method gets the datanode replication count for the current HLog.
1472    *
1473    * If the pipeline isn't started yet or is empty, you will get the default
1474    * replication factor.  Therefore, if this function returns 0, it means you
1475    * are not properly running with the HDFS-826 patch.
1476    * @throws InvocationTargetException
1477    * @throws IllegalAccessException
1478    * @throws IllegalArgumentException
1479    *
1480    * @throws Exception
1481    */
1482   int getLogReplication()
1483   throws IllegalArgumentException, IllegalAccessException, InvocationTargetException {
1484     if (this.getNumCurrentReplicas != null && this.hdfs_out != null) {
1485       Object repl = this.getNumCurrentReplicas.invoke(getOutputStream(), NO_ARGS);
1486       if (repl instanceof Integer) {
1487         return ((Integer)repl).intValue();
1488       }
1489     }
1490     return 0;
1491   }
1492 
1493   boolean canGetCurReplicas() {
1494     return this.getNumCurrentReplicas != null;
1495   }
1496 
1497   @Override
1498   public void hsync() throws IOException {
1499     TraceScope scope = Trace.startSpan("FSHLog.hsync");
1500     try {
1501       scope = Trace.continueSpan(publishSyncThenBlockOnCompletion(scope.detach()));
1502     } finally {
1503       assert scope == NullScope.INSTANCE || !scope.isDetached();
1504       scope.close();
1505     }
1506   }
1507 
1508   @Override
1509   public void hflush() throws IOException {
1510     TraceScope scope = Trace.startSpan("FSHLog.hflush");
1511     try {
1512       scope = Trace.continueSpan(publishSyncThenBlockOnCompletion(scope.detach()));
1513     } finally {
1514       assert scope == NullScope.INSTANCE || !scope.isDetached();
1515       scope.close();
1516     }
1517   }
1518 
1519   @Override
1520   public void sync() throws IOException {
1521     TraceScope scope = Trace.startSpan("FSHLog.sync");
1522     try {
1523       scope = Trace.continueSpan(publishSyncThenBlockOnCompletion(scope.detach()));
1524     } finally {
1525       assert scope == NullScope.INSTANCE || !scope.isDetached();
1526       scope.close();
1527     }
1528   }
1529 
1530   @Override
1531   public void sync(long txid) throws IOException {
1532     if (this.highestSyncedSequence.get() >= txid){
1533       // Already sync'd.
1534       return;
1535     }
1536     TraceScope scope = Trace.startSpan("FSHLog.sync");
1537     try {
1538       scope = Trace.continueSpan(publishSyncThenBlockOnCompletion(scope.detach()));
1539     } finally {
1540       assert scope == NullScope.INSTANCE || !scope.isDetached();
1541       scope.close();
1542     }
1543   }
1544 
1545   void requestLogRoll() {
1546     if (!this.listeners.isEmpty()) {
1547       for (WALActionsListener i: this.listeners) {
1548         i.logRollRequested();
1549       }
1550     }
1551   }
1552 
1553   /** @return How many items have been added to the log */
1554   int getNumEntries() {
1555     return numEntries.get();
1556   }
1557 
1558   /** @return the number of rolled log files */
1559   public int getNumRolledLogFiles() {
1560     return byWalRegionSequenceIds.size();
1561   }
1562 
1563   /** @return the number of log files in use */
1564   @Override
1565   public int getNumLogFiles() {
1566     // +1 for current use log
1567     return getNumRolledLogFiles() + 1;
1568   }
1569   
1570   /** @return the size of log files in use */
1571   @Override
1572   public long getLogFileSize() {
1573     return this.totalLogSize.get();
1574   }
1575   
1576   @Override
1577   public boolean startCacheFlush(final byte[] encodedRegionName) {
1578     Long oldRegionSeqNum = null;
1579     if (!closeBarrier.beginOp()) {
1580       LOG.info("Flush will not be started for " + Bytes.toString(encodedRegionName) +
1581         " - because the server is closing.");
1582       return false;
1583     }
1584     synchronized (regionSequenceIdLock) {
1585       oldRegionSeqNum = this.oldestUnflushedRegionSequenceIds.remove(encodedRegionName);
1586       if (oldRegionSeqNum != null) {
1587         Long oldValue =
1588           this.lowestFlushingRegionSequenceIds.put(encodedRegionName, oldRegionSeqNum);
1589         assert oldValue ==
1590           null : "Flushing map not cleaned up for " + Bytes.toString(encodedRegionName);
1591       }
1592     }
1593     if (oldRegionSeqNum == null) {
1594       // TODO: if we have no oldRegionSeqNum, and WAL is not disabled, presumably either
1595       //       the region is already flushing (which would make this call invalid), or there
1596       //       were no appends after last flush, so why are we starting flush? Maybe we should
1597       //       assert not null, and switch to "long" everywhere. Less rigorous, but safer,
1598       //       alternative is telling the caller to stop. For now preserve old logic.
1599       LOG.warn("Couldn't find oldest seqNum for the region we are about to flush: ["
1600         + Bytes.toString(encodedRegionName) + "]");
1601     }
1602     return true;
1603   }
1604 
1605   @Override
1606   public void completeCacheFlush(final byte [] encodedRegionName) {
1607     synchronized (regionSequenceIdLock) {
1608       this.lowestFlushingRegionSequenceIds.remove(encodedRegionName);
1609     }
1610     closeBarrier.endOp();
1611   }
1612 
1613   @Override
1614   public void abortCacheFlush(byte[] encodedRegionName) {
1615     Long currentSeqNum = null, seqNumBeforeFlushStarts = null;
1616     synchronized (regionSequenceIdLock) {
1617       seqNumBeforeFlushStarts = this.lowestFlushingRegionSequenceIds.remove(encodedRegionName);
1618       if (seqNumBeforeFlushStarts != null) {
1619         currentSeqNum =
1620           this.oldestUnflushedRegionSequenceIds.put(encodedRegionName, seqNumBeforeFlushStarts);
1621       }
1622     }
1623     closeBarrier.endOp();
1624     if ((currentSeqNum != null)
1625         && (currentSeqNum.longValue() <= seqNumBeforeFlushStarts.longValue())) {
1626       String errorStr = "Region " + Bytes.toString(encodedRegionName) +
1627           "acquired edits out of order current memstore seq=" + currentSeqNum
1628           + ", previous oldest unflushed id=" + seqNumBeforeFlushStarts;
1629       LOG.error(errorStr);
1630       assert false : errorStr;
1631       Runtime.getRuntime().halt(1);
1632     }
1633   }
1634 
1635   @Override
1636   public boolean isLowReplicationRollEnabled() {
1637       return lowReplicationRollEnabled;
1638   }
1639 
1640   /**
1641    * Get the directory we are making logs in.
1642    *
1643    * @return dir
1644    */
1645   protected Path getDir() {
1646     return fullPathLogDir;
1647   }
1648 
1649   static Path getHLogArchivePath(Path oldLogDir, Path p) {
1650     return new Path(oldLogDir, p.getName());
1651   }
1652 
1653   static String formatRecoveredEditsFileName(final long seqid) {
1654     return String.format("%019d", seqid);
1655   }
1656 
1657   public static final long FIXED_OVERHEAD = ClassSize.align(
1658     ClassSize.OBJECT + (5 * ClassSize.REFERENCE) +
1659     ClassSize.ATOMIC_INTEGER + Bytes.SIZEOF_INT + (3 * Bytes.SIZEOF_LONG));
1660 
1661   private static void split(final Configuration conf, final Path p)
1662   throws IOException {
1663     FileSystem fs = FileSystem.get(conf);
1664     if (!fs.exists(p)) {
1665       throw new FileNotFoundException(p.toString());
1666     }
1667     if (!fs.getFileStatus(p).isDirectory()) {
1668       throw new IOException(p + " is not a directory");
1669     }
1670 
1671     final Path baseDir = FSUtils.getRootDir(conf);
1672     final Path oldLogDir = new Path(baseDir, HConstants.HREGION_OLDLOGDIR_NAME);
1673     HLogSplitter.split(baseDir, p, oldLogDir, fs, conf);
1674   }
1675 
1676   @Override
1677   public WALCoprocessorHost getCoprocessorHost() {
1678     return coprocessorHost;
1679   }
1680 
1681   @Override
1682   public long getEarliestMemstoreSeqNum(byte[] encodedRegionName) {
1683     Long result = oldestUnflushedRegionSequenceIds.get(encodedRegionName);
1684     return result == null ? HConstants.NO_SEQNUM : result.longValue();
1685   }
1686 
1687   /**
1688    * This class is used coordinating two threads holding one thread at a
1689    * 'safe point' while the orchestrating thread does some work that requires the first thread
1690    * paused: e.g. holding the WAL writer while its WAL is swapped out from under it by another
1691    * thread.
1692    * 
1693    * <p>Thread A signals Thread B to hold when it gets to a 'safe point'.  Thread A wait until
1694    * Thread B gets there. When the 'safe point' has been attained, Thread B signals Thread A.
1695    * Thread B then holds at the 'safe point'.  Thread A on notification that Thread B is paused,
1696    * goes ahead and does the work it needs to do while Thread B is holding.  When Thread A is done,
1697    * it flags B and then Thread A and Thread B continue along on their merry way.  Pause and
1698    * signalling 'zigzags' between the two participating threads.  We use two latches -- one the
1699    * inverse of the other -- pausing and signaling when states are achieved.
1700    * 
1701    * <p>To start up the drama, Thread A creates an instance of this class each time it would do
1702    * this zigzag dance and passes it to Thread B (these classes use Latches so it is one shot
1703    * only). Thread B notices the new instance (via reading a volatile reference or how ever) and it
1704    * starts to work toward the 'safe point'.  Thread A calls {@link #waitSafePoint()} when it
1705    * cannot proceed until the Thread B 'safe point' is attained. Thread A will be held inside in
1706    * {@link #waitSafePoint()} until Thread B reaches the 'safe point'.  Once there, Thread B
1707    * frees Thread A by calling {@link #safePointAttained()}.  Thread A now knows Thread B
1708    * is at the 'safe point' and that it is holding there (When Thread B calls
1709    * {@link #safePointAttained()} it blocks here until Thread A calls {@link #releaseSafePoint()}).
1710    * Thread A proceeds to do what it needs to do while Thread B is paused.  When finished,
1711    * it lets Thread B lose by calling {@link #releaseSafePoint()} and away go both Threads again.
1712    */
1713   static class SafePointZigZagLatch {
1714     /**
1715      * Count down this latch when safe point attained.
1716      */
1717     private volatile CountDownLatch safePointAttainedLatch = new CountDownLatch(1);
1718     /**
1719      * Latch to wait on.  Will be released when we can proceed.
1720      */
1721     private volatile CountDownLatch safePointReleasedLatch = new CountDownLatch(1);
1722  
1723     /**
1724      * For Thread A to call when it is ready to wait on the 'safe point' to be attained.
1725      * Thread A will be held in here until Thread B calls {@link #safePointAttained()}
1726      * @throws InterruptedException
1727      * @throws ExecutionException
1728      * @param syncFuture We need this as barometer on outstanding syncs.  If it comes home with
1729      * an exception, then something is up w/ our syncing.
1730      * @return The passed <code>syncFuture</code>
1731      * @throws FailedSyncBeforeLogCloseException 
1732      */
1733     SyncFuture waitSafePoint(final SyncFuture syncFuture)
1734     throws InterruptedException, FailedSyncBeforeLogCloseException {
1735       while (true) {
1736         if (this.safePointAttainedLatch.await(1, TimeUnit.NANOSECONDS)) break;
1737         if (syncFuture.isThrowable()) {
1738           throw new FailedSyncBeforeLogCloseException(syncFuture.getThrowable());
1739         }
1740       }
1741       return syncFuture;
1742     }
1743  
1744     /**
1745      * Called by Thread B when it attains the 'safe point'.  In this method, Thread B signals
1746      * Thread A it can proceed. Thread B will be held in here until {@link #releaseSafePoint()}
1747      * is called by Thread A.
1748      * @throws InterruptedException
1749      */
1750     void safePointAttained() throws InterruptedException {
1751       this.safePointAttainedLatch.countDown();
1752       this.safePointReleasedLatch.await();
1753     }
1754 
1755     /**
1756      * Called by Thread A when it is done with the work it needs to do while Thread B is
1757      * halted.  This will release the Thread B held in a call to {@link #safePointAttained()}
1758      */
1759     void releaseSafePoint() {
1760       this.safePointReleasedLatch.countDown();
1761     }
1762 
1763     /**
1764      * @return True is this is a 'cocked', fresh instance, and not one that has already fired.
1765      */
1766     boolean isCocked() {
1767       return this.safePointAttainedLatch.getCount() > 0 &&
1768         this.safePointReleasedLatch.getCount() > 0;
1769     }
1770   }
1771 
1772   /**
1773    * Handler that is run by the disruptor ringbuffer consumer. Consumer is a SINGLE
1774    * 'writer/appender' thread.  Appends edits and starts up sync runs.  Tries its best to batch up
1775    * syncs.  There is no discernible benefit batching appends so we just append as they come in
1776    * because it simplifies the below implementation.  See metrics for batching effectiveness
1777    * (In measurement, at 100 concurrent handlers writing 1k, we are batching > 10 appends and 10
1778    * handler sync invocations for every actual dfsclient sync call; at 10 concurrent handlers,
1779    * YMMV).
1780    * <p>Herein, we have an array into which we store the sync futures as they come in.  When we
1781    * have a 'batch', we'll then pass what we have collected to a SyncRunner thread to do the
1782    * filesystem sync.  When it completes, it will then call
1783    * {@link SyncFuture#done(long, Throwable)} on each of SyncFutures in the batch to release
1784    * blocked Handler threads.
1785    * <p>I've tried various effects to try and make latencies low while keeping throughput high.
1786    * I've tried keeping a single Queue of SyncFutures in this class appending to its tail as the
1787    * syncs coming and having sync runner threads poll off the head to 'finish' completed
1788    * SyncFutures.  I've tried linkedlist, and various from concurrent utils whether
1789    * LinkedBlockingQueue or ArrayBlockingQueue, etc.  The more points of synchronization, the
1790    * more 'work' (according to 'perf stats') that has to be done; small increases in stall
1791    * percentages seem to have a big impact on throughput/latencies.  The below model where we have
1792    * an array into which we stash the syncs and then hand them off to the sync thread seemed like
1793    * a decent compromise.  See HBASE-8755 for more detail.
1794    */
1795   class RingBufferEventHandler implements EventHandler<RingBufferTruck>, LifecycleAware {
1796     private final SyncRunner [] syncRunners;
1797     private final SyncFuture [] syncFutures;
1798     // Had 'interesting' issues when this was non-volatile.  On occasion, we'd not pass all
1799     // syncFutures to the next sync'ing thread.
1800     private volatile int syncFuturesCount = 0;
1801     private volatile SafePointZigZagLatch zigzagLatch;
1802     /**
1803      * Object to block on while waiting on safe point.
1804      */
1805     private final Object safePointWaiter = new Object();
1806     private volatile boolean shutdown = false;
1807 
1808     /**
1809      * Which syncrunner to use next.
1810      */
1811     private int syncRunnerIndex;
1812 
1813     RingBufferEventHandler(final int syncRunnerCount, final int maxHandlersCount) {
1814       this.syncFutures = new SyncFuture[maxHandlersCount];
1815       this.syncRunners = new SyncRunner[syncRunnerCount];
1816       for (int i = 0; i < syncRunnerCount; i++) {
1817         this.syncRunners[i] = new SyncRunner("sync." + i, maxHandlersCount);
1818       }
1819     }
1820 
1821     private void cleanupOutstandingSyncsOnException(final long sequence, final Exception e) {
1822       for (int i = 0; i < this.syncFuturesCount; i++) this.syncFutures[i].done(sequence, e);
1823       this.syncFuturesCount = 0;
1824     }
1825 
1826     @Override
1827     // We can set endOfBatch in the below method if at end of our this.syncFutures array
1828     public void onEvent(final RingBufferTruck truck, final long sequence, boolean endOfBatch)
1829     throws Exception {
1830       // Appends and syncs are coming in order off the ringbuffer.  We depend on this fact.  We'll
1831       // add appends to dfsclient as they come in.  Batching appends doesn't give any significant
1832       // benefit on measurement.  Handler sync calls we will batch up.
1833 
1834       try {
1835         if (truck.hasSyncFuturePayload()) {
1836           this.syncFutures[this.syncFuturesCount++] = truck.unloadSyncFuturePayload();
1837           // Force flush of syncs if we are carrying a full complement of syncFutures.
1838           if (this.syncFuturesCount == this.syncFutures.length) endOfBatch = true;
1839         } else if (truck.hasFSWALEntryPayload()) {
1840           TraceScope scope = Trace.continueSpan(truck.unloadSpanPayload());
1841           try {
1842             append(truck.unloadFSWALEntryPayload());
1843           } catch (Exception e) {
1844             // If append fails, presume any pending syncs will fail too; let all waiting handlers
1845             // know of the exception.
1846             cleanupOutstandingSyncsOnException(sequence, e);
1847             // Return to keep processing.
1848             return;
1849           } finally {
1850             assert scope == NullScope.INSTANCE || !scope.isDetached();
1851             scope.close(); // append scope is complete
1852           }
1853         } else {
1854           // They can't both be null.  Fail all up to this!!!
1855           cleanupOutstandingSyncsOnException(sequence,
1856             new IllegalStateException("Neither append nor sync"));
1857           // Return to keep processing.
1858           return;
1859         }
1860 
1861         // TODO: Check size and if big go ahead and call a sync if we have enough data.
1862 
1863         // If not a batch, return to consume more events from the ring buffer before proceeding;
1864         // we want to get up a batch of syncs and appends before we go do a filesystem sync.
1865         if (!endOfBatch || this.syncFuturesCount <= 0) return;
1866 
1867         // Now we have a batch.
1868 
1869         if (LOG.isTraceEnabled()) {
1870           LOG.trace("Sequence=" + sequence + ", syncCount=" + this.syncFuturesCount);
1871         }
1872 
1873         // Below expects that the offer 'transfers' responsibility for the outstanding syncs to the
1874         // syncRunner.
1875         int index = Math.abs(this.syncRunnerIndex++) % this.syncRunners.length;
1876         this.syncRunners[index].offer(sequence, this.syncFutures, this.syncFuturesCount);
1877         attainSafePoint(sequence);
1878         this.syncFuturesCount = 0;
1879       } catch (Throwable t) {
1880         LOG.error("UNEXPECTED!!! syncFutures.length=" + this.syncFutures.length, t);
1881       }
1882     }
1883 
1884     SafePointZigZagLatch attainSafePoint() {
1885       this.zigzagLatch = new SafePointZigZagLatch();
1886       return this.zigzagLatch;
1887     }
1888 
1889     /**
1890      * Check if we should attain safe point.  If so, go there and then wait till signalled before
1891      * we proceeding.
1892      */
1893     private void attainSafePoint(final long currentSequence) {
1894       if (this.zigzagLatch == null || !this.zigzagLatch.isCocked()) return;
1895       // If here, another thread is waiting on us to get to safe point.  Don't leave it hanging.
1896       try {
1897         // Wait on outstanding syncers; wait for them to finish syncing (unless we've been
1898         // shutdown or unless our latch has been thrown because we have been aborted).
1899         while (!this.shutdown && this.zigzagLatch.isCocked() &&
1900             highestSyncedSequence.get() < currentSequence) {
1901           synchronized (this.safePointWaiter) {
1902             this.safePointWaiter.wait(0, 1);
1903           }
1904         }
1905         // Tell waiting thread we've attained safe point
1906         this.zigzagLatch.safePointAttained();
1907       } catch (InterruptedException e) {
1908         LOG.warn("Interrupted ", e);
1909         Thread.currentThread().interrupt();
1910       }
1911     }
1912 
1913     /**
1914      * Append to the WAL.  Does all CP and WAL listener calls.
1915      * @param entry
1916      * @throws Exception
1917      */
1918     void append(final FSWALEntry entry) throws Exception {
1919       // TODO: WORK ON MAKING THIS APPEND FASTER. DOING WAY TOO MUCH WORK WITH CPs, PBing, etc.
1920       atHeadOfRingBufferEventHandlerAppend();
1921 
1922       long start = EnvironmentEdgeManager.currentTimeMillis();
1923       byte [] encodedRegionName = entry.getKey().getEncodedRegionName();
1924       long regionSequenceId = HLog.NO_SEQUENCE_ID;
1925       try {
1926         // We are about to append this edit; update the region-scoped sequence number.  Do it
1927         // here inside this single appending/writing thread.  Events are ordered on the ringbuffer
1928         // so region sequenceids will also be in order.
1929         regionSequenceId = entry.stampRegionSequenceId();
1930         
1931         // Edits are empty, there is nothing to append.  Maybe empty when we are looking for a 
1932         // region sequence id only, a region edit/sequence id that is not associated with an actual 
1933         // edit. It has to go through all the rigmarole to be sure we have the right ordering.
1934         if (entry.getEdit().isEmpty()) {
1935           return;
1936         }
1937         
1938         // Coprocessor hook.
1939         if (!coprocessorHost.preWALWrite(entry.getHRegionInfo(), entry.getKey(),
1940             entry.getEdit())) {
1941           if (entry.getEdit().isReplay()) {
1942             // Set replication scope null so that this won't be replicated
1943             entry.getKey().setScopes(null);
1944           }
1945         }
1946         if (!listeners.isEmpty()) {
1947           for (WALActionsListener i: listeners) {
1948             // TODO: Why does listener take a table description and CPs take a regioninfo?  Fix.
1949             i.visitLogEntryBeforeWrite(entry.getHTableDescriptor(), entry.getKey(),
1950               entry.getEdit());
1951           }
1952         }
1953 
1954         writer.append(entry);
1955         assert highestUnsyncedSequence < entry.getSequence();
1956         highestUnsyncedSequence = entry.getSequence();
1957         Long lRegionSequenceId = Long.valueOf(regionSequenceId);
1958         highestRegionSequenceIds.put(encodedRegionName, lRegionSequenceId);
1959         if (entry.isInMemstore()) {
1960           oldestUnflushedRegionSequenceIds.putIfAbsent(encodedRegionName, lRegionSequenceId);
1961         }
1962         
1963         coprocessorHost.postWALWrite(entry.getHRegionInfo(), entry.getKey(), entry.getEdit());
1964         // Update metrics.
1965         postAppend(entry, EnvironmentEdgeManager.currentTimeMillis() - start);
1966       } catch (Exception e) {
1967         LOG.fatal("Could not append. Requesting close of hlog", e);
1968         requestLogRoll();
1969         throw e;
1970       }
1971       numEntries.incrementAndGet();
1972     }
1973 
1974     @Override
1975     public void onStart() {
1976       for (SyncRunner syncRunner: this.syncRunners) syncRunner.start();
1977     }
1978 
1979     @Override
1980     public void onShutdown() {
1981       for (SyncRunner syncRunner: this.syncRunners) syncRunner.interrupt();
1982     }
1983   }
1984 
1985   /**
1986    * Exposed for testing only.  Use to tricks like halt the ring buffer appending.
1987    */
1988   @VisibleForTesting
1989   void atHeadOfRingBufferEventHandlerAppend() {
1990     // Noop
1991   }
1992 
1993   private static IOException ensureIOException(final Throwable t) {
1994     return (t instanceof IOException)? (IOException)t: new IOException(t);
1995   }
1996 
1997   private static void usage() {
1998     System.err.println("Usage: HLog <ARGS>");
1999     System.err.println("Arguments:");
2000     System.err.println(" --dump  Dump textual representation of passed one or more files");
2001     System.err.println("         For example: " +
2002       "FSHLog --dump hdfs://example.com:9000/hbase/.logs/MACHINE/LOGFILE");
2003     System.err.println(" --split Split the passed directory of WAL logs");
2004     System.err.println("         For example: " +
2005       "FSHLog --split hdfs://example.com:9000/hbase/.logs/DIR");
2006     System.err.println(" --perf  Write the same key <N> times to a WAL: e.g. FSHLog --perf 10");
2007   }
2008 
2009   /**
2010    * Pass one or more log file names and it will either dump out a text version
2011    * on <code>stdout</code> or split the specified log files.
2012    *
2013    * @param args
2014    * @throws IOException
2015    */
2016   public static void main(String[] args) throws IOException {
2017     if (args.length < 2) {
2018       usage();
2019       System.exit(-1);
2020     }
2021     // either dump using the HLogPrettyPrinter or split, depending on args
2022     if (args[0].compareTo("--dump") == 0) {
2023       HLogPrettyPrinter.run(Arrays.copyOfRange(args, 1, args.length));
2024     } else if (args[0].compareTo("--perf") == 0) {
2025       final int count = Integer.parseInt(args[1]);
2026       // Put up a WAL and just keep adding same edit to it.  Simple perf test.
2027       Configuration conf = HBaseConfiguration.create();
2028       Path rootDir = FSUtils.getRootDir(conf);
2029       FileSystem fs = rootDir.getFileSystem(conf);
2030       FSHLog wal =
2031         new FSHLog(fs, rootDir, "perflog", "oldPerflog", conf, null, false, "perf", false);
2032       long start = System.nanoTime();
2033       WALEdit walEdit = new WALEdit();
2034       walEdit.add(new KeyValue(Bytes.toBytes("row"), Bytes.toBytes("family"),
2035         Bytes.toBytes("qualifier"), -1, new byte [1000]));
2036       for (AtomicLong i = new AtomicLong(0); i.get() < count; i.incrementAndGet()) {
2037         wal.append(HRegionInfo.FIRST_META_REGIONINFO, TableName.META_TABLE_NAME, walEdit, start,
2038           HTableDescriptor.META_TABLEDESC, i);
2039         wal.sync();
2040       }
2041       wal.close();
2042       LOG.info("Write " + count + " 1k edits in " + (System.nanoTime() - start) + "nanos");
2043     } else if (args[0].compareTo("--split") == 0) {
2044       Configuration conf = HBaseConfiguration.create();
2045       for (int i = 1; i < args.length; i++) {
2046         try {
2047           Path logPath = new Path(args[i]);
2048           FSUtils.setFsDefault(conf, logPath);
2049           split(conf, logPath);
2050         } catch (IOException t) {
2051           t.printStackTrace(System.err);
2052           System.exit(-1);
2053         }
2054       }
2055     } else {
2056       usage();
2057       System.exit(-1);
2058     }
2059   }
2060   
2061   /**
2062    * Find the 'getPipeline' on the passed <code>os</code> stream.
2063    * @return Method or null.
2064    */
2065   private Method getGetPipeline(final FSDataOutputStream os) {
2066     Method m = null;
2067     if (os != null) {
2068       Class<? extends OutputStream> wrappedStreamClass = os.getWrappedStream()
2069           .getClass();
2070       try {
2071         m = wrappedStreamClass.getDeclaredMethod("getPipeline",
2072           new Class<?>[] {});
2073         m.setAccessible(true);
2074       } catch (NoSuchMethodException e) {
2075         LOG.info("FileSystem's output stream doesn't support"
2076             + " getPipeline; not available; fsOut="
2077             + wrappedStreamClass.getName());
2078       } catch (SecurityException e) {
2079         LOG.info(
2080           "Doesn't have access to getPipeline on "
2081               + "FileSystems's output stream ; fsOut="
2082               + wrappedStreamClass.getName(), e);
2083         m = null; // could happen on setAccessible()
2084       }
2085     }
2086     return m;
2087   }
2088 
2089   /**
2090    * This method gets the pipeline for the current HLog.
2091    * @return
2092    */
2093   DatanodeInfo[] getPipeLine() {
2094     if (this.getPipeLine != null && this.hdfs_out != null) {
2095       Object repl;
2096       try {
2097         repl = this.getPipeLine.invoke(getOutputStream(), NO_ARGS);
2098         if (repl instanceof DatanodeInfo[]) {
2099           return ((DatanodeInfo[]) repl);
2100         }
2101       } catch (Exception e) {
2102         LOG.info("Get pipeline failed", e);
2103       }
2104     }
2105     return new DatanodeInfo[0];
2106   }
2107 }