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