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