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