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