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