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