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