View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.regionserver.wal;
21  
22  import java.io.DataInput;
23  import java.io.DataOutput;
24  import java.io.FileNotFoundException;
25  import java.io.IOException;
26  import java.io.OutputStream;
27  import java.io.UnsupportedEncodingException;
28  import java.lang.reflect.InvocationTargetException;
29  import java.lang.reflect.Method;
30  import java.net.URLEncoder;
31  import java.util.ArrayList;
32  import java.util.Arrays;
33  import java.util.Collections;
34  import java.util.LinkedList;
35  import java.util.List;
36  import java.util.Map;
37  import java.util.NavigableSet;
38  import java.util.SortedMap;
39  import java.util.TreeMap;
40  import java.util.TreeSet;
41  import java.util.UUID;
42  import java.util.concurrent.ConcurrentSkipListMap;
43  import java.util.concurrent.CopyOnWriteArrayList;
44  import java.util.concurrent.atomic.AtomicBoolean;
45  import java.util.concurrent.atomic.AtomicInteger;
46  import java.util.concurrent.atomic.AtomicLong;
47  import java.util.concurrent.locks.Lock;
48  import java.util.concurrent.locks.ReentrantLock;
49  import java.util.regex.Matcher;
50  import java.util.regex.Pattern;
51  
52  import org.apache.commons.logging.Log;
53  import org.apache.commons.logging.LogFactory;
54  import org.apache.hadoop.conf.Configuration;
55  import org.apache.hadoop.fs.FSDataOutputStream;
56  import org.apache.hadoop.fs.FileStatus;
57  import org.apache.hadoop.fs.FileSystem;
58  import org.apache.hadoop.fs.Path;
59  import org.apache.hadoop.fs.PathFilter;
60  import org.apache.hadoop.fs.Syncable;
61  import org.apache.hadoop.hbase.HBaseConfiguration;
62  import org.apache.hadoop.hbase.HConstants;
63  import org.apache.hadoop.hbase.HRegionInfo;
64  import org.apache.hadoop.hbase.HTableDescriptor;
65  import org.apache.hadoop.hbase.KeyValue;
66  import org.apache.hadoop.hbase.ServerName;
67  import org.apache.hadoop.hbase.util.Bytes;
68  import org.apache.hadoop.hbase.util.ClassSize;
69  import org.apache.hadoop.hbase.util.FSUtils;
70  import org.apache.hadoop.hbase.util.HasThread;
71  import org.apache.hadoop.hbase.util.Threads;
72  import org.apache.hadoop.io.Writable;
73  import org.apache.hadoop.util.StringUtils;
74  
75  /**
76   * HLog stores all the edits to the HStore.  Its the hbase write-ahead-log
77   * implementation.
78   *
79   * It performs logfile-rolling, so external callers are not aware that the
80   * underlying file is being rolled.
81   *
82   * <p>
83   * There is one HLog per RegionServer.  All edits for all Regions carried by
84   * a particular RegionServer are entered first in the HLog.
85   *
86   * <p>
87   * Each HRegion is identified by a unique long <code>int</code>. HRegions do
88   * not need to declare themselves before using the HLog; they simply include
89   * their HRegion-id in the <code>append</code> or
90   * <code>completeCacheFlush</code> calls.
91   *
92   * <p>
93   * An HLog consists of multiple on-disk files, which have a chronological order.
94   * As data is flushed to other (better) on-disk structures, the log becomes
95   * obsolete. We can destroy all the log messages for a given HRegion-id up to
96   * the most-recent CACHEFLUSH message from that HRegion.
97   *
98   * <p>
99   * It's only practical to delete entire files. Thus, we delete an entire on-disk
100  * file F when all of the messages in F have a log-sequence-id that's older
101  * (smaller) than the most-recent CACHEFLUSH message for every HRegion that has
102  * a message in F.
103  *
104  * <p>
105  * Synchronized methods can never execute in parallel. However, between the
106  * start of a cache flush and the completion point, appends are allowed but log
107  * rolling is not. To prevent log rolling taking place during this period, a
108  * separate reentrant lock is used.
109  *
110  * <p>To read an HLog, call {@link #getReader(org.apache.hadoop.fs.FileSystem,
111  * org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration)}.
112  *
113  */
114 public class HLog implements Syncable {
115   static final Log LOG = LogFactory.getLog(HLog.class);
116   public static final byte [] METAFAMILY = Bytes.toBytes("METAFAMILY");
117   static final byte [] METAROW = Bytes.toBytes("METAROW");
118 
119   /** File Extension used while splitting an HLog into regions (HBASE-2312) */
120   public static final String SPLITTING_EXT = "-splitting";
121   public static final boolean SPLIT_SKIP_ERRORS_DEFAULT = false;
122 
123   /*
124    * Name of directory that holds recovered edits written by the wal log
125    * splitting code, one per region
126    */
127   public static final String RECOVERED_EDITS_DIR = "recovered.edits";
128   private static final Pattern EDITFILES_NAME_PATTERN =
129     Pattern.compile("-?[0-9]+");
130   public static final String RECOVERED_LOG_TMPFILE_SUFFIX = ".temp";
131   
132   private final FileSystem fs;
133   private final Path dir;
134   private final Configuration conf;
135   // Listeners that are called on WAL events.
136   private List<WALActionsListener> listeners =
137     new CopyOnWriteArrayList<WALActionsListener>();
138   private final long optionalFlushInterval;
139   private final long blocksize;
140   private final String prefix;
141   private final AtomicLong unflushedEntries = new AtomicLong(0);
142   private volatile long syncedTillHere = 0;
143   private long lastDeferredTxid;
144   private final Path oldLogDir;
145   private volatile boolean logRollRunning;
146 
147   private static Class<? extends Writer> logWriterClass;
148   private static Class<? extends Reader> logReaderClass;
149 
150   private WALCoprocessorHost coprocessorHost;
151 
152   static void resetLogReaderClass() {
153     HLog.logReaderClass = null;
154   }
155 
156   private FSDataOutputStream hdfs_out; // FSDataOutputStream associated with the current SequenceFile.writer
157   // Minimum tolerable replicas, if the actual value is lower than it, 
158   // rollWriter will be triggered
159   private int minTolerableReplication;
160   private Method getNumCurrentReplicas; // refers to DFSOutputStream.getNumCurrentReplicas
161   final static Object [] NO_ARGS = new Object []{};
162 
163   public interface Reader {
164     void init(FileSystem fs, Path path, Configuration c) throws IOException;
165     void close() throws IOException;
166     Entry next() throws IOException;
167     Entry next(Entry reuse) throws IOException;
168     void seek(long pos) throws IOException;
169     long getPosition() throws IOException;
170     void reset() throws IOException;
171   }
172 
173   public interface Writer {
174     void init(FileSystem fs, Path path, Configuration c) throws IOException;
175     void close() throws IOException;
176     void sync() throws IOException;
177     void append(Entry entry) throws IOException;
178     long getLength() throws IOException;
179   }
180 
181   /*
182    * Current log file.
183    */
184   Writer writer;
185 
186   /*
187    * Map of all log files but the current one.
188    */
189   final SortedMap<Long, Path> outputfiles =
190     Collections.synchronizedSortedMap(new TreeMap<Long, Path>());
191 
192   /*
193    * Map of encoded region names to their most recent sequence/edit id in their
194    * memstore.
195    */
196   private final ConcurrentSkipListMap<byte [], Long> lastSeqWritten =
197     new ConcurrentSkipListMap<byte [], Long>(Bytes.BYTES_COMPARATOR);
198 
199   private volatile boolean closed = false;
200 
201   private final AtomicLong logSeqNum = new AtomicLong(0);
202 
203   // The timestamp (in ms) when the log file was created.
204   private volatile long filenum = -1;
205 
206   //number of transactions in the current Hlog.
207   private final AtomicInteger numEntries = new AtomicInteger(0);
208 
209   // If live datanode count is lower than the default replicas value,
210   // RollWriter will be triggered in each sync(So the RollWriter will be
211   // triggered one by one in a short time). Using it as a workaround to slow
212   // down the roll frequency triggered by checkLowReplication().
213   private volatile int consecutiveLogRolls = 0;
214   private final int lowReplicationRollLimit;
215 
216   // If consecutiveLogRolls is larger than lowReplicationRollLimit,
217   // then disable the rolling in checkLowReplication().
218   // Enable it if the replications recover.
219   private volatile boolean lowReplicationRollEnabled = true;
220 
221   // If > than this size, roll the log. This is typically 0.95 times the size
222   // of the default Hdfs block size.
223   private final long logrollsize;
224 
225   // This lock prevents starting a log roll during a cache flush.
226   // synchronized is insufficient because a cache flush spans two method calls.
227   private final Lock cacheFlushLock = new ReentrantLock();
228 
229   // We synchronize on updateLock to prevent updates and to prevent a log roll
230   // during an update
231   // locked during appends
232   private final Object updateLock = new Object();
233   private final Object flushLock = new Object();
234 
235   private final boolean enabled;
236 
237   /*
238    * If more than this many logs, force flush of oldest region to oldest edit
239    * goes to disk.  If too many and we crash, then will take forever replaying.
240    * Keep the number of logs tidy.
241    */
242   private final int maxLogs;
243 
244   /**
245    * Thread that handles optional sync'ing
246    */
247   private final LogSyncer logSyncer;
248 
249   /** Number of log close errors tolerated before we abort */
250   private final int closeErrorsTolerated;
251 
252   private final AtomicInteger closeErrorCount = new AtomicInteger();
253 
254   /**
255    * Pattern used to validate a HLog file name
256    */
257   private static final Pattern pattern = Pattern.compile(".*\\.\\d*");
258 
259   static byte [] COMPLETE_CACHE_FLUSH;
260   static {
261     try {
262       COMPLETE_CACHE_FLUSH =
263         "HBASE::CACHEFLUSH".getBytes(HConstants.UTF8_ENCODING);
264     } catch (UnsupportedEncodingException e) {
265       assert(false);
266     }
267   }
268 
269   public static class Metric {
270     public long min = Long.MAX_VALUE;
271     public long max = 0;
272     public long total = 0;
273     public int count = 0;
274 
275     synchronized void inc(final long val) {
276       min = Math.min(min, val);
277       max = Math.max(max, val);
278       total += val;
279       ++count;
280     }
281 
282     synchronized Metric get() {
283       Metric copy = new Metric();
284       copy.min = min;
285       copy.max = max;
286       copy.total = total;
287       copy.count = count;
288       this.min = Long.MAX_VALUE;
289       this.max = 0;
290       this.total = 0;
291       this.count = 0;
292       return copy;
293     }
294   }
295 
296   // For measuring latency of writes
297   private static Metric writeTime = new Metric();
298   private static Metric writeSize = new Metric();
299   // For measuring latency of syncs
300   private static Metric syncTime = new Metric();
301   //For measuring slow HLog appends
302   private static AtomicLong slowHLogAppendCount = new AtomicLong();
303   private static Metric slowHLogAppendTime = new Metric();
304   
305   public static Metric getWriteTime() {
306     return writeTime.get();
307   }
308 
309   public static Metric getWriteSize() {
310     return writeSize.get();
311   }
312 
313   public static Metric getSyncTime() {
314     return syncTime.get();
315   }
316 
317   public static long getSlowAppendCount() {
318     return slowHLogAppendCount.get();
319   }
320   
321   public static Metric getSlowAppendTime() {
322     return slowHLogAppendTime.get();
323   }
324 
325   /**
326    * Constructor.
327    *
328    * @param fs filesystem handle
329    * @param dir path to where hlogs are stored
330    * @param oldLogDir path to where hlogs are archived
331    * @param conf configuration to use
332    * @throws IOException
333    */
334   public HLog(final FileSystem fs, final Path dir, final Path oldLogDir,
335               final Configuration conf)
336   throws IOException {
337     this(fs, dir, oldLogDir, conf, null, true, null);
338   }
339 
340   /**
341    * Create an edit log at the given <code>dir</code> location.
342    *
343    * You should never have to load an existing log. If there is a log at
344    * startup, it should have already been processed and deleted by the time the
345    * HLog object is started up.
346    *
347    * @param fs filesystem handle
348    * @param dir path to where hlogs are stored
349    * @param oldLogDir path to where hlogs are archived
350    * @param conf configuration to use
351    * @param listeners Listeners on WAL events. Listeners passed here will
352    * be registered before we do anything else; e.g. the
353    * Constructor {@link #rollWriter()}.
354    * @param prefix should always be hostname and port in distributed env and
355    *        it will be URL encoded before being used.
356    *        If prefix is null, "hlog" will be used
357    * @throws IOException
358    */
359   public HLog(final FileSystem fs, final Path dir, final Path oldLogDir,
360       final Configuration conf, final List<WALActionsListener> listeners,
361       final String prefix) throws IOException {
362     this(fs, dir, oldLogDir, conf, listeners, true, prefix);
363   }
364 
365   /**
366    * Create an edit log at the given <code>dir</code> location.
367    *
368    * You should never have to load an existing log. If there is a log at
369    * startup, it should have already been processed and deleted by the time the
370    * HLog object is started up.
371    *
372    * @param fs filesystem handle
373    * @param dir path to where hlogs are stored
374    * @param oldLogDir path to where hlogs are archived
375    * @param conf configuration to use
376    * @param listeners Listeners on WAL events. Listeners passed here will
377    * be registered before we do anything else; e.g. the
378    * Constructor {@link #rollWriter()}.
379    * @param failIfLogDirExists If true IOException will be thrown if dir already exists.
380    * @param prefix should always be hostname and port in distributed env and
381    *        it will be URL encoded before being used.
382    *        If prefix is null, "hlog" will be used
383    * @throws IOException
384    */
385   public HLog(final FileSystem fs, final Path dir, final Path oldLogDir,
386       final Configuration conf, final List<WALActionsListener> listeners,
387       final boolean failIfLogDirExists, final String prefix)
388   throws IOException {
389     super();
390     this.fs = fs;
391     this.dir = dir;
392     this.conf = conf;
393     if (listeners != null) {
394       for (WALActionsListener i: listeners) {
395         registerWALActionsListener(i);
396       }
397     }
398     this.blocksize = conf.getLong("hbase.regionserver.hlog.blocksize",
399         getDefaultBlockSize());
400     // Roll at 95% of block size.
401     float multi = conf.getFloat("hbase.regionserver.logroll.multiplier", 0.95f);
402     this.logrollsize = (long)(this.blocksize * multi);
403     this.optionalFlushInterval =
404       conf.getLong("hbase.regionserver.optionallogflushinterval", 1 * 1000);
405     if (failIfLogDirExists && fs.exists(dir)) {
406       throw new IOException("Target HLog directory already exists: " + dir);
407     }
408     if (!fs.mkdirs(dir)) {
409       throw new IOException("Unable to mkdir " + dir);
410     }
411     this.oldLogDir = oldLogDir;
412     if (!fs.exists(oldLogDir)) {
413       if (!fs.mkdirs(this.oldLogDir)) {
414         throw new IOException("Unable to mkdir " + this.oldLogDir);
415       }
416     }
417     this.maxLogs = conf.getInt("hbase.regionserver.maxlogs", 32);
418     this.minTolerableReplication = conf.getInt(
419         "hbase.regionserver.hlog.tolerable.lowreplication",
420         this.fs.getDefaultReplication());
421     this.lowReplicationRollLimit = conf.getInt(
422         "hbase.regionserver.hlog.lowreplication.rolllimit", 5);
423     this.enabled = conf.getBoolean("hbase.regionserver.hlog.enabled", true);
424     this.closeErrorsTolerated = conf.getInt(
425         "hbase.regionserver.logroll.errors.tolerated", 0);
426 
427     LOG.info("HLog configuration: blocksize=" +
428       StringUtils.byteDesc(this.blocksize) +
429       ", rollsize=" + StringUtils.byteDesc(this.logrollsize) +
430       ", enabled=" + this.enabled +
431       ", optionallogflushinternal=" + this.optionalFlushInterval + "ms");
432     // If prefix is null||empty then just name it hlog
433     this.prefix = prefix == null || prefix.isEmpty() ?
434         "hlog" : URLEncoder.encode(prefix, "UTF8");
435     // rollWriter sets this.hdfs_out if it can.
436     rollWriter();
437 
438     // handle the reflection necessary to call getNumCurrentReplicas()
439     this.getNumCurrentReplicas = getGetNumCurrentReplicas(this.hdfs_out);
440 
441     logSyncer = new LogSyncer(this.optionalFlushInterval);
442     // When optionalFlushInterval is set as 0, don't start a thread for deferred log sync.
443     if (this.optionalFlushInterval > 0) {
444       Threads.setDaemonThreadRunning(logSyncer.getThread(), Thread.currentThread().getName()
445           + ".logSyncer");
446     } else {
447       LOG.info("hbase.regionserver.optionallogflushinterval is set as "
448           + this.optionalFlushInterval + ". Deferred log syncing won't work. "
449           + "Any Mutation, marked to be deferred synced, will be flushed immediately.");
450     }
451     coprocessorHost = new WALCoprocessorHost(this, conf);
452   }
453   
454   // use reflection to search for getDefaultBlockSize(Path f)
455   // if the method doesn't exist, fall back to using getDefaultBlockSize()
456   private long getDefaultBlockSize() throws IOException {
457     Method m = null;
458     Class<? extends FileSystem> cls = this.fs.getClass();
459     try {
460       m = cls.getMethod("getDefaultBlockSize",
461           new Class<?>[] { Path.class });
462     } catch (NoSuchMethodException e) {
463       LOG.info("FileSystem doesn't support getDefaultBlockSize");
464     } catch (SecurityException e) {
465       LOG.info("Doesn't have access to getDefaultBlockSize on "
466           + "FileSystems", e);
467       m = null; // could happen on setAccessible()
468     }
469     if (null == m) {
470       return this.fs.getDefaultBlockSize();
471     } else {
472       try {
473         Object ret = m.invoke(this.fs, this.dir);
474         return ((Long)ret).longValue();
475       } catch (Exception e) {
476         throw new IOException(e);
477       }
478     }
479   }
480 
481   /**
482    * Find the 'getNumCurrentReplicas' on the passed <code>os</code> stream.
483    * @return Method or null.
484    */
485   private Method getGetNumCurrentReplicas(final FSDataOutputStream os) {
486     Method m = null;
487     if (os != null) {
488       Class<? extends OutputStream> wrappedStreamClass = os.getWrappedStream()
489           .getClass();
490       try {
491         m = wrappedStreamClass.getDeclaredMethod("getNumCurrentReplicas",
492             new Class<?>[] {});
493         m.setAccessible(true);
494       } catch (NoSuchMethodException e) {
495         LOG.info("FileSystem's output stream doesn't support"
496             + " getNumCurrentReplicas; --HDFS-826 not available; fsOut="
497             + wrappedStreamClass.getName());
498       } catch (SecurityException e) {
499         LOG.info("Doesn't have access to getNumCurrentReplicas on "
500             + "FileSystems's output stream --HDFS-826 not available; fsOut="
501             + wrappedStreamClass.getName(), e);
502         m = null; // could happen on setAccessible()
503       }
504     }
505     if (m != null) {
506       LOG.info("Using getNumCurrentReplicas--HDFS-826");
507     }
508     return m;
509   }
510 
511   public void registerWALActionsListener(final WALActionsListener listener) {
512     this.listeners.add(listener);
513   }
514 
515   public boolean unregisterWALActionsListener(final WALActionsListener listener) {
516     return this.listeners.remove(listener);
517   }
518 
519   /**
520    * @return Current state of the monotonically increasing file id.
521    */
522   public long getFilenum() {
523     return this.filenum;
524   }
525 
526   /**
527    * Called by HRegionServer when it opens a new region to ensure that log
528    * sequence numbers are always greater than the latest sequence number of the
529    * region being brought on-line.
530    *
531    * @param newvalue We'll set log edit/sequence number to this value if it
532    * is greater than the current value.
533    */
534   public void setSequenceNumber(final long newvalue) {
535     for (long id = this.logSeqNum.get(); id < newvalue &&
536         !this.logSeqNum.compareAndSet(id, newvalue); id = this.logSeqNum.get()) {
537       // This could spin on occasion but better the occasional spin than locking
538       // every increment of sequence number.
539       LOG.debug("Changed sequenceid from " + logSeqNum + " to " + newvalue);
540     }
541   }
542 
543   /**
544    * @return log sequence number
545    */
546   public long getSequenceNumber() {
547     return logSeqNum.get();
548   }
549 
550   /**
551    * Method used internal to this class and for tests only.
552    * @return The wrapped stream our writer is using; its not the
553    * writer's 'out' FSDatoOutputStream but the stream that this 'out' wraps
554    * (In hdfs its an instance of DFSDataOutputStream).
555    */
556   // usage: see TestLogRolling.java
557   OutputStream getOutputStream() {
558     return this.hdfs_out.getWrappedStream();
559   }
560 
561   /**
562    * Roll the log writer. That is, start writing log messages to a new file.
563    *
564    * Because a log cannot be rolled during a cache flush, and a cache flush
565    * spans two method calls, a special lock needs to be obtained so that a cache
566    * flush cannot start when the log is being rolled and the log cannot be
567    * rolled during a cache flush.
568    *
569    * <p>Note that this method cannot be synchronized because it is possible that
570    * startCacheFlush runs, obtaining the cacheFlushLock, then this method could
571    * start which would obtain the lock on this but block on obtaining the
572    * cacheFlushLock and then completeCacheFlush could be called which would wait
573    * for the lock on this and consequently never release the cacheFlushLock
574    *
575    * @return If lots of logs, flush the returned regions so next time through
576    * we can clean logs. Returns null if nothing to flush.  Names are actual
577    * region names as returned by {@link HRegionInfo#getEncodedName()}
578    * @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException
579    * @throws IOException
580    */
581   public byte [][] rollWriter() throws FailedLogCloseException, IOException {
582     return rollWriter(false);
583   }
584 
585   /**
586    * Roll the log writer. That is, start writing log messages to a new file.
587    *
588    * Because a log cannot be rolled during a cache flush, and a cache flush
589    * spans two method calls, a special lock needs to be obtained so that a cache
590    * flush cannot start when the log is being rolled and the log cannot be
591    * rolled during a cache flush.
592    *
593    * <p>Note that this method cannot be synchronized because it is possible that
594    * startCacheFlush runs, obtaining the cacheFlushLock, then this method could
595    * start which would obtain the lock on this but block on obtaining the
596    * cacheFlushLock and then completeCacheFlush could be called which would wait
597    * for the lock on this and consequently never release the cacheFlushLock
598    *
599    * @param force If true, force creation of a new writer even if no entries
600    * have been written to the current writer
601    * @return If lots of logs, flush the returned regions so next time through
602    * we can clean logs. Returns null if nothing to flush.  Names are actual
603    * region names as returned by {@link HRegionInfo#getEncodedName()}
604    * @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException
605    * @throws IOException
606    */
607   public byte [][] rollWriter(boolean force)
608       throws FailedLogCloseException, IOException {
609     // Return if nothing to flush.
610     if (!force && this.writer != null && this.numEntries.get() <= 0) {
611       return null;
612     }
613     byte [][] regionsToFlush = null;
614     this.cacheFlushLock.lock();
615     this.logRollRunning = true;
616     try {
617       if (closed) {
618         LOG.debug("HLog closed.  Skipping rolling of writer");
619         return regionsToFlush;
620       }
621       // Do all the preparation outside of the updateLock to block
622       // as less as possible the incoming writes
623       long currentFilenum = this.filenum;
624       Path oldPath = null;
625       if (currentFilenum > 0) {
626         oldPath = computeFilename(currentFilenum);
627       }
628       this.filenum = System.currentTimeMillis();
629       Path newPath = computeFilename();
630 
631       // Tell our listeners that a new log is about to be created
632       if (!this.listeners.isEmpty()) {
633         for (WALActionsListener i : this.listeners) {
634           i.preLogRoll(oldPath, newPath);
635         }
636       }
637       HLog.Writer nextWriter = this.createWriterInstance(fs, newPath, conf);
638       // Can we get at the dfsclient outputstream?  If an instance of
639       // SFLW, it'll have done the necessary reflection to get at the
640       // protected field name.
641       FSDataOutputStream nextHdfsOut = null;
642       if (nextWriter instanceof SequenceFileLogWriter) {
643         nextHdfsOut = ((SequenceFileLogWriter)nextWriter).getWriterFSDataOutputStream();
644       }
645 
646       synchronized (updateLock) {
647         // Clean up current writer.
648         Path oldFile = cleanupCurrentWriter(currentFilenum);
649         this.writer = nextWriter;
650         this.hdfs_out = nextHdfsOut;
651 
652         LOG.info((oldFile != null?
653             "Roll " + FSUtils.getPath(oldFile) + ", entries=" +
654             this.numEntries.get() +
655             ", filesize=" +
656             this.fs.getFileStatus(oldFile).getLen() + ". ": "") +
657           " for " + FSUtils.getPath(newPath));
658         this.numEntries.set(0);
659       }
660       // Tell our listeners that a new log was created
661       if (!this.listeners.isEmpty()) {
662         for (WALActionsListener i : this.listeners) {
663           i.postLogRoll(oldPath, newPath);
664         }
665       }
666 
667       // Can we delete any of the old log files?
668       if (this.outputfiles.size() > 0) {
669         if (this.lastSeqWritten.isEmpty()) {
670           LOG.debug("Last sequenceid written is empty. Deleting all old hlogs");
671           // If so, then no new writes have come in since all regions were
672           // flushed (and removed from the lastSeqWritten map). Means can
673           // remove all but currently open log file.
674           for (Map.Entry<Long, Path> e : this.outputfiles.entrySet()) {
675             archiveLogFile(e.getValue(), e.getKey());
676           }
677           this.outputfiles.clear();
678         } else {
679           regionsToFlush = cleanOldLogs();
680         }
681       }
682     } finally {
683       this.logRollRunning = false;
684       this.cacheFlushLock.unlock();
685     }
686     return regionsToFlush;
687   }
688 
689   /**
690    * This method allows subclasses to inject different writers without having to
691    * extend other methods like rollWriter().
692    * 
693    * @param fs
694    * @param path
695    * @param conf
696    * @return Writer instance
697    * @throws IOException
698    */
699   protected Writer createWriterInstance(final FileSystem fs, final Path path,
700       final Configuration conf) throws IOException {
701     return createWriter(fs, path, conf);
702   }
703 
704   /**
705    * Get a reader for the WAL.
706    * The proper way to tail a log that can be under construction is to first use this method
707    * to get a reader then call {@link HLog.Reader#reset()} to see the new data. It will also
708    * take care of keeping implementation-specific context (like compression).
709    * @param fs
710    * @param path
711    * @param conf
712    * @return A WAL reader.  Close when done with it.
713    * @throws IOException
714    */
715   public static Reader getReader(final FileSystem fs, final Path path,
716                                  Configuration conf)
717       throws IOException {
718     try {
719 
720       if (logReaderClass == null) {
721 
722         logReaderClass = conf.getClass("hbase.regionserver.hlog.reader.impl",
723             SequenceFileLogReader.class, Reader.class);
724       }
725 
726 
727       HLog.Reader reader = logReaderClass.newInstance();
728       reader.init(fs, path, conf);
729       return reader;
730     } catch (IOException e) {
731       throw e;
732     }
733     catch (Exception e) {
734       throw new IOException("Cannot get log reader", e);
735     }
736   }
737 
738   /**
739    * Get a writer for the WAL.
740    * @param path
741    * @param conf
742    * @return A WAL writer.  Close when done with it.
743    * @throws IOException
744    */
745   public static Writer createWriter(final FileSystem fs,
746       final Path path, Configuration conf)
747   throws IOException {
748     try {
749       if (logWriterClass == null) {
750         logWriterClass = conf.getClass("hbase.regionserver.hlog.writer.impl",
751             SequenceFileLogWriter.class, Writer.class);
752       }
753       HLog.Writer writer = (HLog.Writer) logWriterClass.newInstance();
754       writer.init(fs, path, conf);
755       return writer;
756     } catch (Exception e) {
757       throw new IOException("cannot get log writer", e);
758     }
759   }
760 
761   /*
762    * Clean up old commit logs.
763    * @return If lots of logs, flush the returned region so next time through
764    * we can clean logs. Returns null if nothing to flush.  Returns array of
765    * encoded region names to flush.
766    * @throws IOException
767    */
768   private byte [][] cleanOldLogs() throws IOException {
769     Long oldestOutstandingSeqNum = getOldestOutstandingSeqNum();
770     // Get the set of all log files whose last sequence number is smaller than
771     // the oldest edit's sequence number.
772     TreeSet<Long> sequenceNumbers =
773       new TreeSet<Long>(this.outputfiles.headMap(
774         (Long.valueOf(oldestOutstandingSeqNum.longValue()))).keySet());
775     // Now remove old log files (if any)
776     int logsToRemove = sequenceNumbers.size();
777     if (logsToRemove > 0) {
778       if (LOG.isDebugEnabled()) {
779         // Find associated region; helps debugging.
780         byte [] oldestRegion = getOldestRegion(oldestOutstandingSeqNum);
781         LOG.debug("Found " + logsToRemove + " hlogs to remove" +
782           " out of total " + this.outputfiles.size() + ";" +
783           " oldest outstanding sequenceid is " + oldestOutstandingSeqNum +
784           " from region " + Bytes.toStringBinary(oldestRegion));
785       }
786       for (Long seq : sequenceNumbers) {
787         archiveLogFile(this.outputfiles.remove(seq), seq);
788       }
789     }
790 
791     // If too many log files, figure which regions we need to flush.
792     // Array is an array of encoded region names.
793     byte [][] regions = null;
794     int logCount = this.outputfiles == null? 0: this.outputfiles.size();
795     if (logCount > this.maxLogs && logCount > 0) {
796       // This is an array of encoded region names.
797       regions = findMemstoresWithEditsEqualOrOlderThan(this.outputfiles.firstKey(),
798         this.lastSeqWritten);
799       if (regions != null) {
800         StringBuilder sb = new StringBuilder();
801         for (int i = 0; i < regions.length; i++) {
802           if (i > 0) sb.append(", ");
803           sb.append(Bytes.toStringBinary(regions[i]));
804         }
805         LOG.info("Too many hlogs: logs=" + logCount + ", maxlogs=" +
806            this.maxLogs + "; forcing flush of " + regions.length + " regions(s): " +
807            sb.toString());
808       }
809     }
810     return regions;
811   }
812 
813   /**
814    * Return regions (memstores) that have edits that are equal or less than
815    * the passed <code>oldestWALseqid</code>.
816    * @param oldestWALseqid
817    * @param regionsToSeqids Encoded region names to sequence ids
818    * @return All regions whose seqid is < than <code>oldestWALseqid</code> (Not
819    * necessarily in order).  Null if no regions found.
820    */
821   static byte [][] findMemstoresWithEditsEqualOrOlderThan(final long oldestWALseqid,
822       final Map<byte [], Long> regionsToSeqids) {
823     //  This method is static so it can be unit tested the easier.
824     List<byte []> regions = null;
825     for (Map.Entry<byte [], Long> e: regionsToSeqids.entrySet()) {
826       if (e.getValue().longValue() <= oldestWALseqid) {
827         if (regions == null) regions = new ArrayList<byte []>();
828         // Key is encoded region name.
829         regions.add(e.getKey());
830       }
831     }
832     return regions == null?
833       null: regions.toArray(new byte [][] {HConstants.EMPTY_BYTE_ARRAY});
834   }
835 
836   /*
837    * @return Logs older than this id are safe to remove.
838    */
839   private Long getOldestOutstandingSeqNum() {
840     return Collections.min(this.lastSeqWritten.values());
841   }
842 
843   /**
844    * @param oldestOutstandingSeqNum
845    * @return (Encoded) name of oldest outstanding region.
846    */
847   private byte [] getOldestRegion(final Long oldestOutstandingSeqNum) {
848     byte [] oldestRegion = null;
849     for (Map.Entry<byte [], Long> e: this.lastSeqWritten.entrySet()) {
850       if (e.getValue().longValue() == oldestOutstandingSeqNum.longValue()) {
851         // Key is encoded region name.
852         oldestRegion = e.getKey();
853         break;
854       }
855     }
856     return oldestRegion;
857   }
858 
859   /*
860    * Cleans up current writer closing and adding to outputfiles.
861    * Presumes we're operating inside an updateLock scope.
862    * @return Path to current writer or null if none.
863    * @throws IOException
864    */
865   Path cleanupCurrentWriter(final long currentfilenum) throws IOException {
866     Path oldFile = null;
867     if (this.writer != null) {
868       // Close the current writer, get a new one.
869       try {
870         // Wait till all current transactions are written to the hlog.
871         // No new transactions can occur because we have the updatelock.
872         if (this.unflushedEntries.get() != this.syncedTillHere) {
873           LOG.debug("cleanupCurrentWriter " +
874                    " waiting for transactions to get synced " +
875                    " total " + this.unflushedEntries.get() +
876                    " synced till here " + syncedTillHere);
877           sync();
878         }
879         this.writer.close();
880         this.writer = null;
881         closeErrorCount.set(0);
882       } catch (IOException e) {
883         LOG.error("Failed close of HLog writer", e);
884         int errors = closeErrorCount.incrementAndGet();
885         if (errors <= closeErrorsTolerated && !hasDeferredEntries()) {
886           LOG.warn("Riding over HLog close failure! error count="+errors);
887         } else {
888           if (hasDeferredEntries()) {
889             LOG.error("Aborting due to unflushed edits in HLog");
890           }
891           // Failed close of log file.  Means we're losing edits.  For now,
892           // shut ourselves down to minimize loss.  Alternative is to try and
893           // keep going.  See HBASE-930.
894           FailedLogCloseException flce =
895             new FailedLogCloseException("#" + currentfilenum);
896           flce.initCause(e);
897           throw flce;
898         }
899       }
900       if (currentfilenum >= 0) {
901         oldFile = computeFilename(currentfilenum);
902         this.outputfiles.put(Long.valueOf(this.logSeqNum.get()), oldFile);
903       }
904     }
905     return oldFile;
906   }
907 
908   private void archiveLogFile(final Path p, final Long seqno) throws IOException {
909     Path newPath = getHLogArchivePath(this.oldLogDir, p);
910     LOG.info("moving old hlog file " + FSUtils.getPath(p) +
911       " whose highest sequenceid is " + seqno + " to " +
912       FSUtils.getPath(newPath));
913 
914     // Tell our listeners that a log is going to be archived.
915     if (!this.listeners.isEmpty()) {
916       for (WALActionsListener i : this.listeners) {
917         i.preLogArchive(p, newPath);
918       }
919     }
920     if (!this.fs.rename(p, newPath)) {
921       throw new IOException("Unable to rename " + p + " to " + newPath);
922     }
923     // Tell our listeners that a log has been archived.
924     if (!this.listeners.isEmpty()) {
925       for (WALActionsListener i : this.listeners) {
926         i.postLogArchive(p, newPath);
927       }
928     }
929   }
930 
931   /**
932    * This is a convenience method that computes a new filename with a given
933    * using the current HLog file-number
934    * @return Path
935    */
936   protected Path computeFilename() {
937     return computeFilename(this.filenum);
938   }
939 
940   /**
941    * This is a convenience method that computes a new filename with a given
942    * file-number.
943    * @param filenum to use
944    * @return Path
945    */
946   protected Path computeFilename(long filenum) {
947     if (filenum < 0) {
948       throw new RuntimeException("hlog file number can't be < 0");
949     }
950     return new Path(dir, prefix + "." + filenum);
951   }
952 
953   /**
954    * Shut down the log and delete the log directory
955    *
956    * @throws IOException
957    */
958   public void closeAndDelete() throws IOException {
959     close();
960     if (!fs.exists(this.dir)) return;
961     FileStatus[] files = fs.listStatus(this.dir);
962     for(FileStatus file : files) {
963 
964       Path p = getHLogArchivePath(this.oldLogDir, file.getPath());
965       // Tell our listeners that a log is going to be archived.
966       if (!this.listeners.isEmpty()) {
967         for (WALActionsListener i : this.listeners) {
968           i.preLogArchive(file.getPath(), p);
969         }
970       }
971 
972       if (!fs.rename(file.getPath(),p)) {
973         throw new IOException("Unable to rename " + file.getPath() + " to " + p);
974       }
975       // Tell our listeners that a log was archived.
976       if (!this.listeners.isEmpty()) {
977         for (WALActionsListener i : this.listeners) {
978           i.postLogArchive(file.getPath(), p);
979         }
980       }
981     }
982     LOG.debug("Moved " + files.length + " log files to " +
983       FSUtils.getPath(this.oldLogDir));
984     if (!fs.delete(dir, true)) {
985       LOG.info("Unable to delete " + dir);
986     }
987   }
988 
989   /**
990    * Shut down the log.
991    *
992    * @throws IOException
993    */
994   public void close() throws IOException {
995     // When optionalFlushInterval is 0, the logSyncer is not started as a Thread.
996     if (this.optionalFlushInterval > 0) {
997       try {
998         logSyncer.close();
999         // Make sure we synced everything
1000         logSyncer.join(this.optionalFlushInterval * 2);
1001       } catch (InterruptedException e) {
1002         LOG.error("Exception while waiting for syncer thread to die", e);
1003       }
1004     }
1005 
1006     cacheFlushLock.lock();
1007     try {
1008       // Tell our listeners that the log is closing
1009       if (!this.listeners.isEmpty()) {
1010         for (WALActionsListener i : this.listeners) {
1011           i.logCloseRequested();
1012         }
1013       }
1014       synchronized (updateLock) {
1015         this.closed = true;
1016         if (LOG.isDebugEnabled()) {
1017           LOG.debug("closing hlog writer in " + this.dir.toString());
1018         }
1019         if (this.writer != null) {
1020           this.writer.close();
1021         }
1022       }
1023     } finally {
1024       cacheFlushLock.unlock();
1025     }
1026   }
1027 
1028   /**
1029    * @param now
1030    * @param regionName
1031    * @param tableName
1032    * @param clusterId
1033    * @return New log key.
1034    */
1035   protected HLogKey makeKey(byte[] regionName, byte[] tableName, long seqnum,
1036       long now, UUID clusterId) {
1037     return new HLogKey(regionName, tableName, seqnum, now, clusterId);
1038   }
1039 
1040 
1041   /** Append an entry to the log.
1042    *
1043    * @param regionInfo
1044    * @param logEdit
1045    * @param logKey
1046    * @param doSync shall we sync after writing the transaction
1047    * @return The txid of this transaction
1048    * @throws IOException
1049    */
1050   public long append(HRegionInfo regionInfo, HLogKey logKey, WALEdit logEdit,
1051                      HTableDescriptor htd, boolean doSync)
1052   throws IOException {
1053     if (this.closed) {
1054       throw new IOException("Cannot append; log is closed");
1055     }
1056     long txid = 0;
1057     synchronized (updateLock) {
1058       long seqNum = obtainSeqNum();
1059       logKey.setLogSeqNum(seqNum);
1060       // The 'lastSeqWritten' map holds the sequence number of the oldest
1061       // write for each region (i.e. the first edit added to the particular
1062       // memstore). When the cache is flushed, the entry for the
1063       // region being flushed is removed if the sequence number of the flush
1064       // is greater than or equal to the value in lastSeqWritten.
1065       this.lastSeqWritten.putIfAbsent(regionInfo.getEncodedNameAsBytes(),
1066         Long.valueOf(seqNum));
1067       doWrite(regionInfo, logKey, logEdit, htd);
1068       txid = this.unflushedEntries.incrementAndGet();
1069       this.numEntries.incrementAndGet();
1070       if (htd.isDeferredLogFlush()) {
1071         lastDeferredTxid = txid;
1072       }
1073     }
1074 
1075     // Sync if catalog region, and if not then check if that table supports
1076     // deferred log flushing
1077     if (doSync &&
1078         (regionInfo.isMetaRegion() ||
1079         !htd.isDeferredLogFlush())) {
1080       // sync txn to file system
1081       this.sync(txid);
1082     }
1083     return txid;
1084   }
1085 
1086   /**
1087    * Only used in tests.
1088    *
1089    * @param info
1090    * @param tableName
1091    * @param edits
1092    * @param now
1093    * @param htd
1094    * @throws IOException
1095    */
1096   public void append(HRegionInfo info, byte [] tableName, WALEdit edits,
1097     final long now, HTableDescriptor htd)
1098   throws IOException {
1099     append(info, tableName, edits, HConstants.DEFAULT_CLUSTER_ID, now, htd);
1100   }
1101 
1102   /**
1103    * Append a set of edits to the log. Log edits are keyed by (encoded)
1104    * regionName, rowname, and log-sequence-id.
1105    *
1106    * Later, if we sort by these keys, we obtain all the relevant edits for a
1107    * given key-range of the HRegion (TODO). Any edits that do not have a
1108    * matching COMPLETE_CACHEFLUSH message can be discarded.
1109    *
1110    * <p>
1111    * Logs cannot be restarted once closed, or once the HLog process dies. Each
1112    * time the HLog starts, it must create a new log. This means that other
1113    * systems should process the log appropriately upon each startup (and prior
1114    * to initializing HLog).
1115    *
1116    * synchronized prevents appends during the completion of a cache flush or for
1117    * the duration of a log roll.
1118    *
1119    * @param info
1120    * @param tableName
1121    * @param edits
1122    * @param clusterId The originating clusterId for this edit (for replication)
1123    * @param now
1124    * @param doSync shall we sync?
1125    * @return txid of this transaction
1126    * @throws IOException
1127    */
1128   private long append(HRegionInfo info, byte [] tableName, WALEdit edits, UUID clusterId,
1129       final long now, HTableDescriptor htd, boolean doSync)
1130     throws IOException {
1131       if (edits.isEmpty()) return this.unflushedEntries.get();;
1132       if (this.closed) {
1133         throw new IOException("Cannot append; log is closed");
1134       }
1135       long txid = 0;
1136       synchronized (this.updateLock) {
1137         long seqNum = obtainSeqNum();
1138         // The 'lastSeqWritten' map holds the sequence number of the oldest
1139         // write for each region (i.e. the first edit added to the particular
1140         // memstore). . When the cache is flushed, the entry for the
1141         // region being flushed is removed if the sequence number of the flush
1142         // is greater than or equal to the value in lastSeqWritten.
1143         // Use encoded name.  Its shorter, guaranteed unique and a subset of
1144         // actual  name.
1145         byte [] encodedRegionName = info.getEncodedNameAsBytes();
1146         this.lastSeqWritten.putIfAbsent(encodedRegionName, seqNum);
1147         HLogKey logKey = makeKey(encodedRegionName, tableName, seqNum, now, clusterId);
1148         doWrite(info, logKey, edits, htd);
1149         this.numEntries.incrementAndGet();
1150         txid = this.unflushedEntries.incrementAndGet();
1151         if (htd.isDeferredLogFlush()) {
1152           lastDeferredTxid = txid;
1153         }
1154       }
1155       // Sync if catalog region, and if not then check if that table supports
1156       // deferred log flushing
1157       if (doSync && 
1158           (info.isMetaRegion() ||
1159           !htd.isDeferredLogFlush())) {
1160         // sync txn to file system
1161         this.sync(txid);
1162       }
1163       return txid;
1164     }
1165 
1166   /**
1167    * Append a set of edits to the log. Log edits are keyed by (encoded)
1168    * regionName, rowname, and log-sequence-id. The HLog is not flushed
1169    * after this transaction is written to the log.
1170    *
1171    * @param info
1172    * @param tableName
1173    * @param edits
1174    * @param clusterId The originating clusterId for this edit (for replication)
1175    * @param now
1176    * @return txid of this transaction
1177    * @throws IOException
1178    */
1179   public long appendNoSync(HRegionInfo info, byte [] tableName, WALEdit edits, 
1180     UUID clusterId, final long now, HTableDescriptor htd)
1181     throws IOException {
1182     return append(info, tableName, edits, clusterId, now, htd, false);
1183   }
1184 
1185   /**
1186    * Append a set of edits to the log. Log edits are keyed by (encoded)
1187    * regionName, rowname, and log-sequence-id. The HLog is flushed
1188    * after this transaction is written to the log.
1189    *
1190    * @param info
1191    * @param tableName
1192    * @param edits
1193    * @param clusterId The originating clusterId for this edit (for replication)
1194    * @param now
1195    * @return txid of this transaction
1196    * @throws IOException
1197    */
1198   public long append(HRegionInfo info, byte [] tableName, WALEdit edits, 
1199     UUID clusterId, final long now, HTableDescriptor htd)
1200     throws IOException {
1201     return append(info, tableName, edits, clusterId, now, htd, true);
1202   }
1203 
1204   /**
1205    * This class is responsible to hold the HLog's appended Entry list
1206    * and to sync them according to a configurable interval.
1207    *
1208    * Deferred log flushing works first by piggy backing on this process by
1209    * simply not sync'ing the appended Entry. It can also be sync'd by other
1210    * non-deferred log flushed entries outside of this thread.
1211    */
1212   class LogSyncer extends HasThread {
1213 
1214     private final long optionalFlushInterval;
1215 
1216     private AtomicBoolean closeLogSyncer = new AtomicBoolean(false);
1217 
1218     // List of pending writes to the HLog. There corresponds to transactions
1219     // that have not yet returned to the client. We keep them cached here
1220     // instead of writing them to HDFS piecemeal, because the HDFS write 
1221     // method is pretty heavyweight as far as locking is concerned. The 
1222     // goal is to increase the batchsize for writing-to-hdfs as well as
1223     // sync-to-hdfs, so that we can get better system throughput.
1224     private List<Entry> pendingWrites = new LinkedList<Entry>();
1225 
1226     LogSyncer(long optionalFlushInterval) {
1227       this.optionalFlushInterval = optionalFlushInterval;
1228     }
1229 
1230     @Override
1231     public void run() {
1232       try {
1233         // awaiting with a timeout doesn't always
1234         // throw exceptions on interrupt
1235         while(!this.isInterrupted() && !closeLogSyncer.get()) {
1236 
1237           try {
1238             if (unflushedEntries.get() <= syncedTillHere) {
1239               synchronized (closeLogSyncer) {
1240                 closeLogSyncer.wait(this.optionalFlushInterval);
1241               }
1242             }
1243             // Calling sync since we waited or had unflushed entries.
1244             // Entries appended but not sync'd are taken care of here AKA
1245             // deferred log flush
1246             sync();
1247           } catch (IOException e) {
1248             LOG.error("Error while syncing, requesting close of hlog ", e);
1249             requestLogRoll();
1250           }
1251         }
1252       } catch (InterruptedException e) {
1253         LOG.debug(getName() + " interrupted while waiting for sync requests");
1254       } finally {
1255         LOG.info(getName() + " exiting");
1256       }
1257     }
1258 
1259     // appends new writes to the pendingWrites. It is better to keep it in
1260     // our own queue rather than writing it to the HDFS output stream because
1261     // HDFSOutputStream.writeChunk is not lightweight at all.
1262     synchronized void append(Entry e) throws IOException {
1263       pendingWrites.add(e);
1264     }
1265 
1266     // Returns all currently pending writes. New writes
1267     // will accumulate in a new list.
1268     synchronized List<Entry> getPendingWrites() {
1269       List<Entry> save = this.pendingWrites;
1270       this.pendingWrites = new LinkedList<Entry>();
1271       return save;
1272     }
1273 
1274     // writes out pending entries to the HLog
1275     void hlogFlush(Writer writer, List<Entry> pending) throws IOException {
1276       if (pending == null) return;
1277 
1278       // write out all accumulated Entries to hdfs.
1279       for (Entry e : pending) {
1280         writer.append(e);
1281       }
1282     }
1283 
1284     void close() {
1285       synchronized (closeLogSyncer) {
1286         closeLogSyncer.set(true);
1287         closeLogSyncer.notifyAll();
1288       }
1289     }
1290   }
1291 
1292   // sync all known transactions
1293   private void syncer() throws IOException {
1294     syncer(this.unflushedEntries.get()); // sync all pending items
1295   }
1296 
1297   // sync all transactions upto the specified txid
1298   private void syncer(long txid) throws IOException {
1299     Writer tempWriter;
1300     synchronized (this.updateLock) {
1301       if (this.closed) return;
1302       tempWriter = this.writer; // guaranteed non-null
1303     }
1304     // if the transaction that we are interested in is already 
1305     // synced, then return immediately.
1306     if (txid <= this.syncedTillHere) {
1307       return;
1308     }
1309     try {
1310       long doneUpto;
1311       long now = System.currentTimeMillis();
1312       // First flush all the pending writes to HDFS. Then 
1313       // issue the sync to HDFS. If sync is successful, then update
1314       // syncedTillHere to indicate that transactions till this
1315       // number has been successfully synced.
1316       IOException ioe = null;
1317       List<Entry> pending = null;
1318       synchronized (flushLock) {
1319         if (txid <= this.syncedTillHere) {
1320           return;
1321         }
1322         doneUpto = this.unflushedEntries.get();
1323         pending = logSyncer.getPendingWrites();
1324         try {
1325           logSyncer.hlogFlush(tempWriter, pending);
1326         } catch(IOException io) {
1327           ioe = io;
1328           LOG.error("syncer encountered error, will retry. txid=" + txid, ioe);
1329         }
1330       }
1331       if (ioe != null && pending != null) {
1332         synchronized (this.updateLock) {
1333           synchronized (flushLock) {
1334             // HBASE-4387, HBASE-5623, retry with updateLock held
1335             tempWriter = this.writer;
1336             logSyncer.hlogFlush(tempWriter, pending);
1337           }
1338         }
1339       }
1340       // another thread might have sync'ed avoid double-sync'ing
1341       if (txid <= this.syncedTillHere) {
1342         return;
1343       }
1344       try {
1345         tempWriter.sync();
1346       } catch (IOException io) {
1347         synchronized (this.updateLock) {
1348           // HBASE-4387, HBASE-5623, retry with updateLock held
1349           tempWriter = this.writer;
1350           tempWriter.sync();
1351         }
1352       }
1353       this.syncedTillHere = Math.max(this.syncedTillHere, doneUpto);
1354 
1355       syncTime.inc(System.currentTimeMillis() - now);
1356       if (!this.logRollRunning) {
1357         checkLowReplication();
1358         try {
1359           if (tempWriter.getLength() > this.logrollsize) {
1360             requestLogRoll();
1361           }
1362         } catch (IOException x) {
1363           LOG.debug("Log roll failed and will be retried. (This is not an error)");
1364         }
1365       }
1366     } catch (IOException e) {
1367       LOG.fatal("Could not sync. Requesting close of hlog", e);
1368       requestLogRoll();
1369       throw e;
1370     }
1371   }
1372 
1373   private void checkLowReplication() {
1374     // if the number of replicas in HDFS has fallen below the configured
1375     // value, then roll logs.
1376     try {
1377       int numCurrentReplicas = getLogReplication();
1378       if (numCurrentReplicas != 0
1379           && numCurrentReplicas < this.minTolerableReplication) {
1380         if (this.lowReplicationRollEnabled) {
1381           if (this.consecutiveLogRolls < this.lowReplicationRollLimit) {
1382             LOG.warn("HDFS pipeline error detected. " + "Found "
1383                 + numCurrentReplicas + " replicas but expecting no less than "
1384                 + this.minTolerableReplication + " replicas. "
1385                 + " Requesting close of hlog.");
1386             requestLogRoll();
1387             // If rollWriter is requested, increase consecutiveLogRolls. Once it
1388             // is larger than lowReplicationRollLimit, disable the
1389             // LowReplication-Roller
1390             this.consecutiveLogRolls++;
1391           } else {
1392             LOG.warn("Too many consecutive RollWriter requests, it's a sign of "
1393                 + "the total number of live datanodes is lower than the tolerable replicas.");
1394             this.consecutiveLogRolls = 0;
1395             this.lowReplicationRollEnabled = false;
1396           }
1397         }
1398       } else if (numCurrentReplicas >= this.minTolerableReplication) {
1399 
1400         if (!this.lowReplicationRollEnabled) {
1401           // The new writer's log replicas is always the default value.
1402           // So we should not enable LowReplication-Roller. If numEntries
1403           // is lower than or equals 1, we consider it as a new writer.
1404           if (this.numEntries.get() <= 1) {
1405             return;
1406           }
1407           // Once the live datanode number and the replicas return to normal,
1408           // enable the LowReplication-Roller.
1409           this.lowReplicationRollEnabled = true;
1410           LOG.info("LowReplication-Roller was enabled.");
1411         }
1412       }
1413     } catch (Exception e) {
1414       LOG.warn("Unable to invoke DFSOutputStream.getNumCurrentReplicas" + e +
1415           " still proceeding ahead...");
1416     }
1417   }
1418 
1419   /**
1420    * This method gets the datanode replication count for the current HLog.
1421    *
1422    * If the pipeline isn't started yet or is empty, you will get the default
1423    * replication factor.  Therefore, if this function returns 0, it means you
1424    * are not properly running with the HDFS-826 patch.
1425    * @throws InvocationTargetException
1426    * @throws IllegalAccessException
1427    * @throws IllegalArgumentException
1428    *
1429    * @throws Exception
1430    */
1431   int getLogReplication()
1432   throws IllegalArgumentException, IllegalAccessException, InvocationTargetException {
1433     if (this.getNumCurrentReplicas != null && this.hdfs_out != null) {
1434       Object repl = this.getNumCurrentReplicas.invoke(getOutputStream(), NO_ARGS);
1435       if (repl instanceof Integer) {
1436         return ((Integer)repl).intValue();
1437       }
1438     }
1439     return 0;
1440   }
1441 
1442   boolean canGetCurReplicas() {
1443     return this.getNumCurrentReplicas != null;
1444   }
1445 
1446   public void hsync() throws IOException {
1447     syncer();
1448   }
1449 
1450   public void hflush() throws IOException {
1451     syncer();
1452   }
1453 
1454   public void sync() throws IOException {
1455     syncer();
1456   }
1457 
1458   public void sync(long txid) throws IOException {
1459     syncer(txid);
1460   }
1461 
1462   private void requestLogRoll() {
1463     if (!this.listeners.isEmpty()) {
1464       for (WALActionsListener i: this.listeners) {
1465         i.logRollRequested();
1466       }
1467     }
1468   }
1469 
1470   protected void doWrite(HRegionInfo info, HLogKey logKey, WALEdit logEdit,
1471                            HTableDescriptor htd)
1472   throws IOException {
1473     if (!this.enabled) {
1474       return;
1475     }
1476     if (!this.listeners.isEmpty()) {
1477       for (WALActionsListener i: this.listeners) {
1478         i.visitLogEntryBeforeWrite(htd, logKey, logEdit);
1479       }
1480     }
1481     try {
1482       long now = System.currentTimeMillis();
1483       // coprocessor hook:
1484       if (!coprocessorHost.preWALWrite(info, logKey, logEdit)) {
1485         // write to our buffer for the Hlog file.
1486         logSyncer.append(new HLog.Entry(logKey, logEdit));
1487       }
1488       long took = System.currentTimeMillis() - now;
1489       coprocessorHost.postWALWrite(info, logKey, logEdit);
1490       writeTime.inc(took);
1491       long len = 0;
1492       for (KeyValue kv : logEdit.getKeyValues()) {
1493         len += kv.getLength();
1494       }
1495       writeSize.inc(len);
1496       if (took > 1000) {
1497         LOG.warn(String.format(
1498           "%s took %d ms appending an edit to hlog; editcount=%d, len~=%s",
1499           Thread.currentThread().getName(), took, this.numEntries.get(),
1500           StringUtils.humanReadableInt(len)));
1501         slowHLogAppendCount.incrementAndGet();
1502         slowHLogAppendTime.inc(took);
1503       }
1504     } catch (IOException e) {
1505       LOG.fatal("Could not append. Requesting close of hlog", e);
1506       requestLogRoll();
1507       throw e;
1508     }
1509   }
1510 
1511 
1512   /** @return How many items have been added to the log */
1513   int getNumEntries() {
1514     return numEntries.get();
1515   }
1516 
1517   /**
1518    * Obtain a log sequence number.
1519    */
1520   private long obtainSeqNum() {
1521     return this.logSeqNum.incrementAndGet();
1522   }
1523 
1524   /** @return the number of log files in use */
1525   int getNumLogFiles() {
1526     return outputfiles.size();
1527   }
1528 
1529   private byte[] getSnapshotName(byte[] encodedRegionName) {
1530     byte snp[] = new byte[encodedRegionName.length + 3];
1531     // an encoded region name has only hex digits. s, n or p are not hex
1532     // and therefore snapshot-names will never collide with
1533     // encoded-region-names
1534     snp[0] = 's'; snp[1] = 'n'; snp[2] = 'p';
1535     for (int i = 0; i < encodedRegionName.length; i++) {
1536       snp[i+3] = encodedRegionName[i];
1537     }
1538     return snp;
1539   }
1540 
1541   /**
1542    * By acquiring a log sequence ID, we can allow log messages to continue while
1543    * we flush the cache.
1544    *
1545    * Acquire a lock so that we do not roll the log between the start and
1546    * completion of a cache-flush. Otherwise the log-seq-id for the flush will
1547    * not appear in the correct logfile.
1548    *
1549    * Ensuring that flushes and log-rolls don't happen concurrently also allows
1550    * us to temporarily put a log-seq-number in lastSeqWritten against the region
1551    * being flushed that might not be the earliest in-memory log-seq-number for
1552    * that region. By the time the flush is completed or aborted and before the
1553    * cacheFlushLock is released it is ensured that lastSeqWritten again has the
1554    * oldest in-memory edit's lsn for the region that was being flushed.
1555    *
1556    * In this method, by removing the entry in lastSeqWritten for the region
1557    * being flushed we ensure that the next edit inserted in this region will be
1558    * correctly recorded in {@link #append(HRegionInfo, byte[], WALEdit, long, HTableDescriptor)} The
1559    * lsn of the earliest in-memory lsn - which is now in the memstore snapshot -
1560    * is saved temporarily in the lastSeqWritten map while the flush is active.
1561    *
1562    * @return sequence ID to pass
1563    *         {@link #completeCacheFlush(byte[], byte[], long, boolean)} (byte[],
1564    *         byte[], long)}
1565    * @see #completeCacheFlush(byte[], byte[], long, boolean)
1566    * @see #abortCacheFlush(byte[])
1567    */
1568   public long startCacheFlush(final byte[] encodedRegionName) {
1569     this.cacheFlushLock.lock();
1570     Long seq = this.lastSeqWritten.remove(encodedRegionName);
1571     // seq is the lsn of the oldest edit associated with this region. If a
1572     // snapshot already exists - because the last flush failed - then seq will
1573     // be the lsn of the oldest edit in the snapshot
1574     if (seq != null) {
1575       // keeping the earliest sequence number of the snapshot in
1576       // lastSeqWritten maintains the correctness of
1577       // getOldestOutstandingSeqNum(). But it doesn't matter really because
1578       // everything is being done inside of cacheFlush lock.
1579       Long oldseq =
1580         lastSeqWritten.put(getSnapshotName(encodedRegionName), seq);
1581       if (oldseq != null) {
1582         LOG.error("Logic Error Snapshot seq id from earlier flush still" +
1583             " present! for region " + Bytes.toString(encodedRegionName) +
1584             " overwritten oldseq=" + oldseq + "with new seq=" + seq);
1585         Runtime.getRuntime().halt(1);
1586       }
1587     }
1588     return obtainSeqNum();
1589   }
1590 
1591 
1592   /**
1593    * Complete the cache flush
1594    *
1595    * Protected by cacheFlushLock
1596    *
1597    * @param encodedRegionName
1598    * @param tableName
1599    * @param logSeqId
1600    * @throws IOException
1601    */
1602   public void completeCacheFlush(final byte [] encodedRegionName,
1603       final byte [] tableName, final long logSeqId, final boolean isMetaRegion)
1604   throws IOException {
1605     try {
1606       if (this.closed) {
1607         return;
1608       }
1609       long txid = 0;
1610       synchronized (updateLock) {
1611         long now = System.currentTimeMillis();
1612         WALEdit edit = completeCacheFlushLogEdit();
1613         HLogKey key = makeKey(encodedRegionName, tableName, logSeqId,
1614             System.currentTimeMillis(), HConstants.DEFAULT_CLUSTER_ID);
1615         logSyncer.append(new Entry(key, edit));
1616         txid = this.unflushedEntries.incrementAndGet();
1617         writeTime.inc(System.currentTimeMillis() - now);
1618         long len = 0;
1619         for (KeyValue kv : edit.getKeyValues()) {
1620           len += kv.getLength();
1621         }
1622         writeSize.inc(len);
1623         this.numEntries.incrementAndGet();
1624       }
1625       // sync txn to file system
1626       this.sync(txid);
1627 
1628     } finally {
1629       // updateLock not needed for removing snapshot's entry
1630       // Cleaning up of lastSeqWritten is in the finally clause because we
1631       // don't want to confuse getOldestOutstandingSeqNum()
1632       this.lastSeqWritten.remove(getSnapshotName(encodedRegionName));
1633       this.cacheFlushLock.unlock();
1634     }
1635   }
1636 
1637   private WALEdit completeCacheFlushLogEdit() {
1638     KeyValue kv = new KeyValue(METAROW, METAFAMILY, null,
1639       System.currentTimeMillis(), COMPLETE_CACHE_FLUSH);
1640     WALEdit e = new WALEdit();
1641     e.add(kv);
1642     return e;
1643   }
1644 
1645   /**
1646    * Abort a cache flush.
1647    * Call if the flush fails. Note that the only recovery for an aborted flush
1648    * currently is a restart of the regionserver so the snapshot content dropped
1649    * by the failure gets restored to the memstore.
1650    */
1651   public void abortCacheFlush(byte[] encodedRegionName) {
1652     Long snapshot_seq =
1653       this.lastSeqWritten.remove(getSnapshotName(encodedRegionName));
1654     if (snapshot_seq != null) {
1655       // updateLock not necessary because we are racing against
1656       // lastSeqWritten.putIfAbsent() in append() and we will always win
1657       // before releasing cacheFlushLock make sure that the region's entry in
1658       // lastSeqWritten points to the earliest edit in the region
1659       Long current_memstore_earliest_seq =
1660         this.lastSeqWritten.put(encodedRegionName, snapshot_seq);
1661       if (current_memstore_earliest_seq != null &&
1662           (current_memstore_earliest_seq.longValue() <=
1663             snapshot_seq.longValue())) {
1664         LOG.error("Logic Error region " + Bytes.toString(encodedRegionName) +
1665             "acquired edits out of order current memstore seq=" +
1666             current_memstore_earliest_seq + " snapshot seq=" + snapshot_seq);
1667         Runtime.getRuntime().halt(1);
1668       }
1669     }
1670     this.cacheFlushLock.unlock();
1671   }
1672 
1673   /**
1674    * @param family
1675    * @return true if the column is a meta column
1676    */
1677   public static boolean isMetaFamily(byte [] family) {
1678     return Bytes.equals(METAFAMILY, family);
1679   }
1680 
1681   /**
1682    * Get LowReplication-Roller status
1683    * 
1684    * @return lowReplicationRollEnabled
1685    */
1686   public boolean isLowReplicationRollEnabled() {
1687     return lowReplicationRollEnabled;
1688   }
1689 
1690   @SuppressWarnings("unchecked")
1691   public static Class<? extends HLogKey> getKeyClass(Configuration conf) {
1692      return (Class<? extends HLogKey>)
1693        conf.getClass("hbase.regionserver.hlog.keyclass", HLogKey.class);
1694   }
1695 
1696   public static HLogKey newKey(Configuration conf) throws IOException {
1697     Class<? extends HLogKey> keyClass = getKeyClass(conf);
1698     try {
1699       return keyClass.newInstance();
1700     } catch (InstantiationException e) {
1701       throw new IOException("cannot create hlog key");
1702     } catch (IllegalAccessException e) {
1703       throw new IOException("cannot create hlog key");
1704     }
1705   }
1706 
1707   /**
1708    * Utility class that lets us keep track of the edit with it's key
1709    * Only used when splitting logs
1710    */
1711   public static class Entry implements Writable {
1712     private WALEdit edit;
1713     private HLogKey key;
1714 
1715     public Entry() {
1716       edit = new WALEdit();
1717       key = new HLogKey();
1718     }
1719 
1720     /**
1721      * Constructor for both params
1722      * @param edit log's edit
1723      * @param key log's key
1724      */
1725     public Entry(HLogKey key, WALEdit edit) {
1726       super();
1727       this.key = key;
1728       this.edit = edit;
1729     }
1730     /**
1731      * Gets the edit
1732      * @return edit
1733      */
1734     public WALEdit getEdit() {
1735       return edit;
1736     }
1737     /**
1738      * Gets the key
1739      * @return key
1740      */
1741     public HLogKey getKey() {
1742       return key;
1743     }
1744 
1745     /**
1746      * Set compression context for this entry.
1747      * @param compressionContext Compression context
1748      */
1749     public void setCompressionContext(CompressionContext compressionContext) {
1750       edit.setCompressionContext(compressionContext);
1751       key.setCompressionContext(compressionContext);
1752     }
1753 
1754     @Override
1755     public String toString() {
1756       return this.key + "=" + this.edit;
1757     }
1758 
1759     @Override
1760     public void write(DataOutput dataOutput) throws IOException {
1761       this.key.write(dataOutput);
1762       this.edit.write(dataOutput);
1763     }
1764 
1765     @Override
1766     public void readFields(DataInput dataInput) throws IOException {
1767       this.key.readFields(dataInput);
1768       this.edit.readFields(dataInput);
1769     }
1770   }
1771 
1772   /**
1773    * Construct the HLog directory name
1774    *
1775    * @param serverName Server name formatted as described in {@link ServerName}
1776    * @return the HLog directory name
1777    */
1778   public static String getHLogDirectoryName(final String serverName) {
1779     StringBuilder dirName = new StringBuilder(HConstants.HREGION_LOGDIR_NAME);
1780     dirName.append("/");
1781     dirName.append(serverName);
1782     return dirName.toString();
1783   }
1784 
1785   /**
1786    * Get the directory we are making logs in.
1787    * 
1788    * @return dir
1789    */
1790   protected Path getDir() {
1791     return dir;
1792   }
1793   
1794   /**
1795    * @param filename name of the file to validate
1796    * @return <tt>true</tt> if the filename matches an HLog, <tt>false</tt>
1797    *         otherwise
1798    */
1799   public static boolean validateHLogFilename(String filename) {
1800     return pattern.matcher(filename).matches();
1801   }
1802 
1803   static Path getHLogArchivePath(Path oldLogDir, Path p) {
1804     return new Path(oldLogDir, p.getName());
1805   }
1806 
1807   static String formatRecoveredEditsFileName(final long seqid) {
1808     return String.format("%019d", seqid);
1809   }
1810 
1811   /**
1812    * Returns sorted set of edit files made by wal-log splitter, excluding files
1813    * with '.temp' suffix.
1814    * @param fs
1815    * @param regiondir
1816    * @return Files in passed <code>regiondir</code> as a sorted set.
1817    * @throws IOException
1818    */
1819   public static NavigableSet<Path> getSplitEditFilesSorted(final FileSystem fs,
1820       final Path regiondir)
1821   throws IOException {
1822     NavigableSet<Path> filesSorted = new TreeSet<Path>();
1823     Path editsdir = getRegionDirRecoveredEditsDir(regiondir);
1824     if (!fs.exists(editsdir)) return filesSorted;
1825     FileStatus[] files = FSUtils.listStatus(fs, editsdir, new PathFilter() {
1826       @Override
1827       public boolean accept(Path p) {
1828         boolean result = false;
1829         try {
1830           // Return files and only files that match the editfile names pattern.
1831           // There can be other files in this directory other than edit files.
1832           // In particular, on error, we'll move aside the bad edit file giving
1833           // it a timestamp suffix.  See moveAsideBadEditsFile.
1834           Matcher m = EDITFILES_NAME_PATTERN.matcher(p.getName());
1835           result = fs.isFile(p) && m.matches();
1836           // Skip the file whose name ends with RECOVERED_LOG_TMPFILE_SUFFIX,
1837           // because it means splithlog thread is writting this file.
1838           if (p.getName().endsWith(RECOVERED_LOG_TMPFILE_SUFFIX)) {
1839             result = false;
1840           }
1841         } catch (IOException e) {
1842           LOG.warn("Failed isFile check on " + p);
1843         }
1844         return result;
1845       }
1846     });
1847     if (files == null) return filesSorted;
1848     for (FileStatus status: files) {
1849       filesSorted.add(status.getPath());
1850     }
1851     return filesSorted;
1852   }
1853 
1854   /**
1855    * Move aside a bad edits file.
1856    * @param fs
1857    * @param edits Edits file to move aside.
1858    * @return The name of the moved aside file.
1859    * @throws IOException
1860    */
1861   public static Path moveAsideBadEditsFile(final FileSystem fs,
1862       final Path edits)
1863   throws IOException {
1864     Path moveAsideName = new Path(edits.getParent(), edits.getName() + "." +
1865       System.currentTimeMillis());
1866     if (!fs.rename(edits, moveAsideName)) {
1867       LOG.warn("Rename failed from " + edits + " to " + moveAsideName);
1868     }
1869     return moveAsideName;
1870   }
1871 
1872   /**
1873    * @param regiondir This regions directory in the filesystem.
1874    * @return The directory that holds recovered edits files for the region
1875    * <code>regiondir</code>
1876    */
1877   public static Path getRegionDirRecoveredEditsDir(final Path regiondir) {
1878     return new Path(regiondir, RECOVERED_EDITS_DIR);
1879   }
1880 
1881   public static final long FIXED_OVERHEAD = ClassSize.align(
1882     ClassSize.OBJECT + (5 * ClassSize.REFERENCE) +
1883     ClassSize.ATOMIC_INTEGER + Bytes.SIZEOF_INT + (3 * Bytes.SIZEOF_LONG));
1884 
1885   private static void usage() {
1886     System.err.println("Usage: HLog <ARGS>");
1887     System.err.println("Arguments:");
1888     System.err.println(" --dump  Dump textual representation of passed one or more files");
1889     System.err.println("         For example: HLog --dump hdfs://example.com:9000/hbase/.logs/MACHINE/LOGFILE");
1890     System.err.println(" --split Split the passed directory of WAL logs");
1891     System.err.println("         For example: HLog --split hdfs://example.com:9000/hbase/.logs/DIR");
1892   }
1893 
1894   private static void split(final Configuration conf, final Path p)
1895   throws IOException {
1896     FileSystem fs = FileSystem.get(conf);
1897     if (!fs.exists(p)) {
1898       throw new FileNotFoundException(p.toString());
1899     }
1900     final Path baseDir = new Path(conf.get(HConstants.HBASE_DIR));
1901     final Path oldLogDir = new Path(baseDir, HConstants.HREGION_OLDLOGDIR_NAME);
1902     if (!fs.getFileStatus(p).isDir()) {
1903       throw new IOException(p + " is not a directory");
1904     }
1905 
1906     HLogSplitter logSplitter = HLogSplitter.createLogSplitter(
1907         conf, baseDir, p, oldLogDir, fs);
1908     logSplitter.splitLog();
1909   }
1910 
1911   /**
1912    * @return Coprocessor host.
1913    */
1914   public WALCoprocessorHost getCoprocessorHost() {
1915     return coprocessorHost;
1916   }
1917 
1918   /** Provide access to currently deferred sequence num for tests */
1919   boolean hasDeferredEntries() {
1920     return lastDeferredTxid > syncedTillHere;
1921   }
1922 
1923   /**
1924    * Pass one or more log file names and it will either dump out a text version
1925    * on <code>stdout</code> or split the specified log files.
1926    *
1927    * @param args
1928    * @throws IOException
1929    */
1930   public static void main(String[] args) throws IOException {
1931     if (args.length < 2) {
1932       usage();
1933       System.exit(-1);
1934     }
1935     // either dump using the HLogPrettyPrinter or split, depending on args
1936     if (args[0].compareTo("--dump") == 0) {
1937       HLogPrettyPrinter.run(Arrays.copyOfRange(args, 1, args.length));
1938     } else if (args[0].compareTo("--split") == 0) {
1939       Configuration conf = HBaseConfiguration.create();
1940       for (int i = 1; i < args.length; i++) {
1941         try {
1942           conf.set("fs.default.name", args[i]);
1943           conf.set("fs.defaultFS", args[i]);
1944           Path logPath = new Path(args[i]);
1945           split(conf, logPath);
1946         } catch (Throwable t) {
1947           t.printStackTrace(System.err);
1948           System.exit(-1);
1949         }
1950       }
1951     } else {
1952       usage();
1953       System.exit(-1);
1954     }
1955   }
1956 }