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