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