View Javadoc

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