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