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 static org.apache.hadoop.hbase.util.FSUtils.recoverFileLease;
23  
24  import java.io.DataInput;
25  import java.io.DataOutput;
26  import java.io.EOFException;
27  import java.io.FileNotFoundException;
28  import java.io.IOException;
29  import java.io.OutputStream;
30  import java.io.UnsupportedEncodingException;
31  import java.lang.reflect.InvocationTargetException;
32  import java.lang.reflect.Method;
33  import java.net.URLEncoder;
34  import java.util.ArrayList;
35  import java.util.Collections;
36  import java.util.HashMap;
37  import java.util.LinkedList;
38  import java.util.List;
39  import java.util.Map;
40  import java.util.NavigableSet;
41  import java.util.SortedMap;
42  import java.util.TreeMap;
43  import java.util.TreeSet;
44  import java.util.concurrent.Callable;
45  import java.util.concurrent.ConcurrentSkipListMap;
46  import java.util.concurrent.CopyOnWriteArrayList;
47  import java.util.concurrent.ExecutionException;
48  import java.util.concurrent.Executors;
49  import java.util.concurrent.Future;
50  import java.util.concurrent.ThreadPoolExecutor;
51  import java.util.concurrent.TimeUnit;
52  import java.util.concurrent.atomic.AtomicInteger;
53  import java.util.concurrent.atomic.AtomicLong;
54  import java.util.concurrent.locks.Condition;
55  import java.util.concurrent.locks.Lock;
56  import java.util.concurrent.locks.ReentrantLock;
57  import java.util.regex.Matcher;
58  import java.util.regex.Pattern;
59  
60  import org.apache.commons.logging.Log;
61  import org.apache.commons.logging.LogFactory;
62  import org.apache.hadoop.conf.Configuration;
63  import org.apache.hadoop.fs.FileStatus;
64  import org.apache.hadoop.fs.FileSystem;
65  import org.apache.hadoop.fs.Path;
66  import org.apache.hadoop.fs.PathFilter;
67  import org.apache.hadoop.fs.Syncable;
68  import org.apache.hadoop.hbase.HBaseConfiguration;
69  import org.apache.hadoop.hbase.HConstants;
70  import org.apache.hadoop.hbase.HRegionInfo;
71  import org.apache.hadoop.hbase.HServerInfo;
72  import org.apache.hadoop.hbase.HTableDescriptor;
73  import org.apache.hadoop.hbase.KeyValue;
74  import org.apache.hadoop.hbase.RemoteExceptionHandler;
75  import org.apache.hadoop.hbase.regionserver.HRegion;
76  import org.apache.hadoop.hbase.util.Bytes;
77  import org.apache.hadoop.hbase.util.ClassSize;
78  import org.apache.hadoop.hbase.util.FSUtils;
79  import org.apache.hadoop.hbase.util.Threads;
80  import org.apache.hadoop.io.Writable;
81  
82  import com.google.common.util.concurrent.NamingThreadFactory;
83  
84  /**
85   * HLog stores all the edits to the HStore.  Its the hbase write-ahead-log
86   * implementation.
87   *
88   * It performs logfile-rolling, so external callers are not aware that the
89   * underlying file is being rolled.
90   *
91   * <p>
92   * There is one HLog per RegionServer.  All edits for all Regions carried by
93   * a particular RegionServer are entered first in the HLog.
94   *
95   * <p>
96   * Each HRegion is identified by a unique long <code>int</code>. HRegions do
97   * not need to declare themselves before using the HLog; they simply include
98   * their HRegion-id in the <code>append</code> or
99   * <code>completeCacheFlush</code> calls.
100  *
101  * <p>
102  * An HLog consists of multiple on-disk files, which have a chronological order.
103  * As data is flushed to other (better) on-disk structures, the log becomes
104  * obsolete. We can destroy all the log messages for a given HRegion-id up to
105  * the most-recent CACHEFLUSH message from that HRegion.
106  *
107  * <p>
108  * It's only practical to delete entire files. Thus, we delete an entire on-disk
109  * file F when all of the messages in F have a log-sequence-id that's older
110  * (smaller) than the most-recent CACHEFLUSH message for every HRegion that has
111  * a message in F.
112  *
113  * <p>
114  * Synchronized methods can never execute in parallel. However, between the
115  * start of a cache flush and the completion point, appends are allowed but log
116  * rolling is not. To prevent log rolling taking place during this period, a
117  * separate reentrant lock is used.
118  *
119  * <p>To read an HLog, call {@link #getReader(org.apache.hadoop.fs.FileSystem,
120  * org.apache.hadoop.fs.Path, org.apache.hadoop.conf.Configuration)}.
121  *
122  */
123 public class HLog implements Syncable {
124   static final Log LOG = LogFactory.getLog(HLog.class);
125   public static final byte [] METAFAMILY = Bytes.toBytes("METAFAMILY");
126   static final byte [] METAROW = Bytes.toBytes("METAROW");
127 
128   /*
129    * Name of directory that holds recovered edits written by the wal log
130    * splitting code, one per region
131    */
132   private static final String RECOVERED_EDITS_DIR = "recovered.edits";
133   private static final Pattern EDITFILES_NAME_PATTERN =
134     Pattern.compile("-?[0-9]+");
135   
136   private final FileSystem fs;
137   private final Path dir;
138   private final Configuration conf;
139   private final LogRollListener listener;
140   private final long optionalFlushInterval;
141   private final long blocksize;
142   private final int flushlogentries;
143   private final String prefix;
144   private final AtomicInteger unflushedEntries = new AtomicInteger(0);
145   private final Path oldLogDir;
146   private final List<LogActionsListener> actionListeners =
147       Collections.synchronizedList(new ArrayList<LogActionsListener>());
148 
149 
150   private static Class<? extends Writer> logWriterClass;
151   private static Class<? extends Reader> logReaderClass;
152 
153   private OutputStream hdfs_out;     // OutputStream associated with the current SequenceFile.writer
154   private int initialReplication;    // initial replication factor of SequenceFile.writer
155   private Method getNumCurrentReplicas; // refers to DFSOutputStream.getNumCurrentReplicas
156   final static Object [] NO_ARGS = new Object []{};
157 
158   // used to indirectly tell syncFs to force the sync
159   private boolean forceSync = false;
160 
161   public interface Reader {
162     void init(FileSystem fs, Path path, Configuration c) throws IOException;
163     void close() throws IOException;
164     Entry next() throws IOException;
165     Entry next(Entry reuse) throws IOException;
166     void seek(long pos) throws IOException;
167     long getPosition() throws IOException;
168   }
169 
170   public interface Writer {
171     void init(FileSystem fs, Path path, Configuration c) throws IOException;
172     void close() throws IOException;
173     void sync() throws IOException;
174     void append(Entry entry) throws IOException;
175     long getLength() throws IOException;
176   }
177 
178   /*
179    * Current log file.
180    */
181   Writer writer;
182 
183   /*
184    * Map of all log files but the current one.
185    */
186   final SortedMap<Long, Path> outputfiles =
187     Collections.synchronizedSortedMap(new TreeMap<Long, Path>());
188 
189   /*
190    * Map of regions to first sequence/edit id in their memstore.
191    */
192   private final ConcurrentSkipListMap<byte [], Long> lastSeqWritten =
193     new ConcurrentSkipListMap<byte [], Long>(Bytes.BYTES_COMPARATOR);
194 
195   private volatile boolean closed = false;
196 
197   private final AtomicLong logSeqNum = new AtomicLong(0);
198 
199   // The timestamp (in ms) when the log file was created.
200   private volatile long filenum = -1;
201 
202   //number of transactions in the current Hlog.
203   private final AtomicInteger numEntries = new AtomicInteger(0);
204 
205   // If > than this size, roll the log. This is typically 0.95 times the size
206   // of the default Hdfs block size.
207   private final long logrollsize;
208 
209   // This lock prevents starting a log roll during a cache flush.
210   // synchronized is insufficient because a cache flush spans two method calls.
211   private final Lock cacheFlushLock = new ReentrantLock();
212 
213   // We synchronize on updateLock to prevent updates and to prevent a log roll
214   // during an update
215   private final Object updateLock = new Object();
216 
217   private final boolean enabled;
218 
219   /*
220    * If more than this many logs, force flush of oldest region to oldest edit
221    * goes to disk.  If too many and we crash, then will take forever replaying.
222    * Keep the number of logs tidy.
223    */
224   private final int maxLogs;
225 
226   /**
227    * Thread that handles group commit
228    */
229   private final LogSyncer logSyncerThread;
230 
231   private final List<LogEntryVisitor> logEntryVisitors =
232       new CopyOnWriteArrayList<LogEntryVisitor>();
233 
234   /**
235    * Pattern used to validate a HLog file name
236    */
237   private static final Pattern pattern = Pattern.compile(".*\\.\\d*");
238 
239   static byte [] COMPLETE_CACHE_FLUSH;
240   static {
241     try {
242       COMPLETE_CACHE_FLUSH =
243         "HBASE::CACHEFLUSH".getBytes(HConstants.UTF8_ENCODING);
244     } catch (UnsupportedEncodingException e) {
245       assert(false);
246     }
247   }
248 
249   // For measuring latency of writes
250   private static volatile long writeOps;
251   private static volatile long writeTime;
252   // For measuring latency of syncs
253   private static volatile long syncOps;
254   private static volatile long syncTime;
255 
256   public static long getWriteOps() {
257     long ret = writeOps;
258     writeOps = 0;
259     return ret;
260   }
261 
262   public static long getWriteTime() {
263     long ret = writeTime;
264     writeTime = 0;
265     return ret;
266   }
267 
268   public static long getSyncOps() {
269     long ret = syncOps;
270     syncOps = 0;
271     return ret;
272   }
273 
274   public static long getSyncTime() {
275     long ret = syncTime;
276     syncTime = 0;
277     return ret;
278   }
279 
280   /**
281    * HLog creating with a null actions listener.
282    *
283    * @param fs filesystem handle
284    * @param dir path to where hlogs are stored
285    * @param oldLogDir path to where hlogs are archived
286    * @param conf configuration to use
287    * @param listener listerner used to request log rolls
288    * @throws IOException
289    */
290   public HLog(final FileSystem fs, final Path dir, final Path oldLogDir,
291               final Configuration conf, final LogRollListener listener)
292   throws IOException {
293     this(fs, dir, oldLogDir, conf, listener, null, null);
294   }
295 
296   /**
297    * Create an edit log at the given <code>dir</code> location.
298    *
299    * You should never have to load an existing log. If there is a log at
300    * startup, it should have already been processed and deleted by the time the
301    * HLog object is started up.
302    *
303    * @param fs filesystem handle
304    * @param dir path to where hlogs are stored
305    * @param oldLogDir path to where hlogs are archived
306    * @param conf configuration to use
307    * @param listener listerner used to request log rolls
308    * @param actionListener optional listener for hlog actions like archiving
309    * @param prefix should always be hostname and port in distributed env and
310    *        it will be URL encoded before being used.
311    *        If prefix is null, "hlog" will be used
312    * @throws IOException
313    */
314   public HLog(final FileSystem fs, final Path dir, final Path oldLogDir,
315               final Configuration conf, final LogRollListener listener,
316               final LogActionsListener actionListener, final String prefix)
317   throws IOException {
318     super();
319     this.fs = fs;
320     this.dir = dir;
321     this.conf = conf;
322     this.listener = listener;
323     this.flushlogentries =
324       conf.getInt("hbase.regionserver.flushlogentries", 1);
325     this.blocksize = conf.getLong("hbase.regionserver.hlog.blocksize",
326       this.fs.getDefaultBlockSize());
327     // Roll at 95% of block size.
328     float multi = conf.getFloat("hbase.regionserver.logroll.multiplier", 0.95f);
329     this.logrollsize = (long)(this.blocksize * multi);
330     this.optionalFlushInterval =
331       conf.getLong("hbase.regionserver.optionallogflushinterval", 1 * 1000);
332     if (fs.exists(dir)) {
333       throw new IOException("Target HLog directory already exists: " + dir);
334     }
335     fs.mkdirs(dir);
336     this.oldLogDir = oldLogDir;
337     if (!fs.exists(oldLogDir)) {
338       fs.mkdirs(this.oldLogDir);
339     }
340     this.maxLogs = conf.getInt("hbase.regionserver.maxlogs", 32);
341     this.enabled = conf.getBoolean("hbase.regionserver.hlog.enabled", true);
342     LOG.info("HLog configuration: blocksize=" + this.blocksize +
343       ", rollsize=" + this.logrollsize +
344       ", enabled=" + this.enabled +
345       ", flushlogentries=" + this.flushlogentries +
346       ", optionallogflushinternal=" + this.optionalFlushInterval + "ms");
347     if (actionListener != null) {
348       addLogActionsListerner(actionListener);
349     }
350     // If prefix is null||empty then just name it hlog
351     this.prefix = prefix == null || prefix.isEmpty() ?
352         "hlog" : URLEncoder.encode(prefix, "UTF8");
353     // rollWriter sets this.hdfs_out if it can.
354     rollWriter();
355 
356     // handle the reflection necessary to call getNumCurrentReplicas()
357     this.getNumCurrentReplicas = null;
358     if(this.hdfs_out != null) {
359       try {
360         this.getNumCurrentReplicas = this.hdfs_out.getClass().
361           getMethod("getNumCurrentReplicas", new Class<?> []{});
362         this.getNumCurrentReplicas.setAccessible(true);
363       } catch (NoSuchMethodException e) {
364         // Thrown if getNumCurrentReplicas() function isn't available
365       } catch (SecurityException e) {
366         // Thrown if we can't get access to getNumCurrentReplicas()
367         this.getNumCurrentReplicas = null; // could happen on setAccessible()
368       }
369     }
370     if(this.getNumCurrentReplicas != null) {
371       LOG.info("Using getNumCurrentReplicas--HDFS-826");
372     } else {
373       LOG.info("getNumCurrentReplicas--HDFS-826 not available" );
374     }
375 
376     logSyncerThread = new LogSyncer(this.optionalFlushInterval);
377     Threads.setDaemonThreadRunning(logSyncerThread,
378         Thread.currentThread().getName() + ".logSyncer");
379   }
380 
381   /**
382    * @return Current state of the monotonically increasing file id.
383    */
384   public long getFilenum() {
385     return this.filenum;
386   }
387 
388   /**
389    * Called by HRegionServer when it opens a new region to ensure that log
390    * sequence numbers are always greater than the latest sequence number of the
391    * region being brought on-line.
392    *
393    * @param newvalue We'll set log edit/sequence number to this value if it
394    * is greater than the current value.
395    */
396   public void setSequenceNumber(final long newvalue) {
397     for (long id = this.logSeqNum.get(); id < newvalue &&
398         !this.logSeqNum.compareAndSet(id, newvalue); id = this.logSeqNum.get()) {
399       // This could spin on occasion but better the occasional spin than locking
400       // every increment of sequence number.
401       LOG.debug("Changed sequenceid from " + logSeqNum + " to " + newvalue);
402     }
403   }
404 
405   /**
406    * @return log sequence number
407    */
408   public long getSequenceNumber() {
409     return logSeqNum.get();
410   }
411 
412   // usage: see TestLogRolling.java
413   OutputStream getOutputStream() {
414     return this.hdfs_out;
415   }
416 
417   /**
418    * Roll the log writer. That is, start writing log messages to a new file.
419    *
420    * Because a log cannot be rolled during a cache flush, and a cache flush
421    * spans two method calls, a special lock needs to be obtained so that a cache
422    * flush cannot start when the log is being rolled and the log cannot be
423    * rolled during a cache flush.
424    *
425    * <p>Note that this method cannot be synchronized because it is possible that
426    * startCacheFlush runs, obtaining the cacheFlushLock, then this method could
427    * start which would obtain the lock on this but block on obtaining the
428    * cacheFlushLock and then completeCacheFlush could be called which would wait
429    * for the lock on this and consequently never release the cacheFlushLock
430    *
431    * @return If lots of logs, flush the returned regions so next time through
432    * we can clean logs. Returns null if nothing to flush.
433    * @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException
434    * @throws IOException
435    */
436   public byte [][] rollWriter() throws FailedLogCloseException, IOException {
437     // Return if nothing to flush.
438     if (this.writer != null && this.numEntries.get() <= 0) {
439       return null;
440     }
441     byte [][] regionsToFlush = null;
442     this.cacheFlushLock.lock();
443     try {
444       if (closed) {
445         return regionsToFlush;
446       }
447       // Do all the preparation outside of the updateLock to block
448       // as less as possible the incoming writes
449       long currentFilenum = this.filenum;
450       this.filenum = System.currentTimeMillis();
451       Path newPath = computeFilename();
452       HLog.Writer nextWriter = createWriter(fs, newPath, HBaseConfiguration.create(conf));
453       int nextInitialReplication = fs.getFileStatus(newPath).getReplication();
454       // Can we get at the dfsclient outputstream?  If an instance of
455       // SFLW, it'll have done the necessary reflection to get at the
456       // protected field name.
457       OutputStream nextHdfsOut = null;
458       if (nextWriter instanceof SequenceFileLogWriter) {
459         nextHdfsOut =
460           ((SequenceFileLogWriter)nextWriter).getDFSCOutputStream();
461       }
462       synchronized (updateLock) {
463         // Clean up current writer.
464         Path oldFile = cleanupCurrentWriter(currentFilenum);
465         this.writer = nextWriter;
466         this.initialReplication = nextInitialReplication;
467         this.hdfs_out = nextHdfsOut;
468 
469         LOG.info((oldFile != null?
470             "Roll " + FSUtils.getPath(oldFile) + ", entries=" +
471             this.numEntries.get() +
472             ", filesize=" +
473             this.fs.getFileStatus(oldFile).getLen() + ". ": "") +
474           "New hlog " + FSUtils.getPath(newPath));
475         this.numEntries.set(0);
476       }
477       // Tell our listeners that a new log was created
478       if (!this.actionListeners.isEmpty()) {
479         for (LogActionsListener list : this.actionListeners) {
480           list.logRolled(newPath);
481         }
482       }
483       // Can we delete any of the old log files?
484       if (this.outputfiles.size() > 0) {
485         if (this.lastSeqWritten.size() <= 0) {
486           LOG.debug("Last sequenceid written is empty. Deleting all old hlogs");
487           // If so, then no new writes have come in since all regions were
488           // flushed (and removed from the lastSeqWritten map). Means can
489           // remove all but currently open log file.
490           for (Map.Entry<Long, Path> e : this.outputfiles.entrySet()) {
491             archiveLogFile(e.getValue(), e.getKey());
492           }
493           this.outputfiles.clear();
494         } else {
495           regionsToFlush = cleanOldLogs();
496         }
497       }
498     } finally {
499       this.cacheFlushLock.unlock();
500     }
501     return regionsToFlush;
502   }
503 
504   /**
505    * Get a reader for the WAL.
506    * @param fs
507    * @param path
508    * @param conf
509    * @return A WAL reader.  Close when done with it.
510    * @throws IOException
511    */
512   public static Reader getReader(final FileSystem fs,
513     final Path path, Configuration conf)
514   throws IOException {
515     try {
516       if (logReaderClass == null) {
517         logReaderClass =conf.getClass("hbase.regionserver.hlog.reader.impl",
518                 SequenceFileLogReader.class, Reader.class);
519       }
520 
521       HLog.Reader reader = logReaderClass.newInstance();
522       reader.init(fs, path, conf);
523       return reader;
524     } catch (IOException e) {
525       throw e;
526     }
527     catch (Exception e) {
528       throw new IOException("Cannot get log reader", e);
529     }
530   }
531 
532   /**
533    * Get a writer for the WAL.
534    * @param path
535    * @param conf
536    * @return A WAL writer.  Close when done with it.
537    * @throws IOException
538    */
539   public static Writer createWriter(final FileSystem fs,
540       final Path path, Configuration conf)
541   throws IOException {
542     try {
543       if (logWriterClass == null) {
544         logWriterClass = conf.getClass("hbase.regionserver.hlog.writer.impl",
545                 SequenceFileLogWriter.class, Writer.class);
546       }
547       HLog.Writer writer = (HLog.Writer) logWriterClass.newInstance();
548       writer.init(fs, path, conf);
549       return writer;
550     } catch (Exception e) {
551       IOException ie = new IOException("cannot get log writer");
552       ie.initCause(e);
553       throw ie;
554     }
555   }
556 
557   /*
558    * Clean up old commit logs.
559    * @return If lots of logs, flush the returned region so next time through
560    * we can clean logs. Returns null if nothing to flush.
561    * @throws IOException
562    */
563   private byte [][] cleanOldLogs() throws IOException {
564     Long oldestOutstandingSeqNum = getOldestOutstandingSeqNum();
565     // Get the set of all log files whose final ID is older than or
566     // equal to the oldest pending region operation
567     TreeSet<Long> sequenceNumbers =
568       new TreeSet<Long>(this.outputfiles.headMap(
569         (Long.valueOf(oldestOutstandingSeqNum.longValue() + 1L))).keySet());
570     // Now remove old log files (if any)
571     int logsToRemove = sequenceNumbers.size();
572     if (logsToRemove > 0) {
573       if (LOG.isDebugEnabled()) {
574         // Find associated region; helps debugging.
575         byte [] oldestRegion = getOldestRegion(oldestOutstandingSeqNum);
576         LOG.debug("Found " + logsToRemove + " hlogs to remove " +
577           " out of total " + this.outputfiles.size() + "; " +
578           "oldest outstanding sequenceid is " + oldestOutstandingSeqNum +
579           " from region " + Bytes.toString(oldestRegion));
580       }
581       for (Long seq : sequenceNumbers) {
582         archiveLogFile(this.outputfiles.remove(seq), seq);
583       }
584     }
585 
586     // If too many log files, figure which regions we need to flush.
587     byte [][] regions = null;
588     int logCount = this.outputfiles.size() - logsToRemove;
589     if (logCount > this.maxLogs && this.outputfiles != null &&
590         this.outputfiles.size() > 0) {
591       regions = findMemstoresWithEditsOlderThan(this.outputfiles.firstKey(),
592         this.lastSeqWritten);
593       StringBuilder sb = new StringBuilder();
594       for (int i = 0; i < regions.length; i++) {
595         if (i > 0) sb.append(", ");
596         sb.append(Bytes.toStringBinary(regions[i]));
597       }
598       LOG.info("Too many hlogs: logs=" + logCount + ", maxlogs=" +
599         this.maxLogs + "; forcing flush of " + regions.length + " regions(s): " +
600         sb.toString());
601     }
602     return regions;
603   }
604 
605   /**
606    * Return regions (memstores) that have edits that are less than the passed
607    * <code>oldestWALseqid</code>.
608    * @param oldestWALseqid
609    * @param regionsToSeqids
610    * @return All regions whose seqid is < than <code>oldestWALseqid</code> (Not
611    * necessarily in order).  Null if no regions found.
612    */
613   static byte [][] findMemstoresWithEditsOlderThan(final long oldestWALseqid,
614       final Map<byte [], Long> regionsToSeqids) {
615     //  This method is static so it can be unit tested the easier.
616     List<byte []> regions = null;
617     for (Map.Entry<byte [], Long> e: regionsToSeqids.entrySet()) {
618       if (e.getValue().longValue() < oldestWALseqid) {
619         if (regions == null) regions = new ArrayList<byte []>();
620         regions.add(e.getKey());
621       }
622     }
623     return regions == null?
624       null: regions.toArray(new byte [][] {HConstants.EMPTY_BYTE_ARRAY});
625   }
626 
627   /*
628    * @return Logs older than this id are safe to remove.
629    */
630   private Long getOldestOutstandingSeqNum() {
631     return Collections.min(this.lastSeqWritten.values());
632   }
633 
634   private byte [] getOldestRegion(final Long oldestOutstandingSeqNum) {
635     byte [] oldestRegion = null;
636     for (Map.Entry<byte [], Long> e: this.lastSeqWritten.entrySet()) {
637       if (e.getValue().longValue() == oldestOutstandingSeqNum.longValue()) {
638         oldestRegion = e.getKey();
639         break;
640       }
641     }
642     return oldestRegion;
643   }
644 
645   /*
646    * Cleans up current writer closing and adding to outputfiles.
647    * Presumes we're operating inside an updateLock scope.
648    * @return Path to current writer or null if none.
649    * @throws IOException
650    */
651   private Path cleanupCurrentWriter(final long currentfilenum)
652   throws IOException {
653     Path oldFile = null;
654     if (this.writer != null) {
655       // Close the current writer, get a new one.
656       try {
657         this.writer.close();
658       } catch (IOException e) {
659         // Failed close of log file.  Means we're losing edits.  For now,
660         // shut ourselves down to minimize loss.  Alternative is to try and
661         // keep going.  See HBASE-930.
662         FailedLogCloseException flce =
663           new FailedLogCloseException("#" + currentfilenum);
664         flce.initCause(e);
665         throw e;
666       }
667       if (currentfilenum >= 0) {
668         oldFile = computeFilename(currentfilenum);
669         this.outputfiles.put(Long.valueOf(this.logSeqNum.get() - 1), oldFile);
670       }
671     }
672     return oldFile;
673   }
674 
675   private void archiveLogFile(final Path p, final Long seqno) throws IOException {
676     Path newPath = getHLogArchivePath(this.oldLogDir, p);
677     LOG.info("moving old hlog file " + FSUtils.getPath(p) +
678       " whose highest sequenceid is " + seqno + " to " +
679       FSUtils.getPath(newPath));
680     this.fs.rename(p, newPath);
681   }
682 
683   /**
684    * This is a convenience method that computes a new filename with a given
685    * using the current HLog file-number
686    * @return Path
687    */
688   protected Path computeFilename() {
689     return computeFilename(this.filenum);
690   }
691 
692   /**
693    * This is a convenience method that computes a new filename with a given
694    * file-number.
695    * @param file-number to use
696    * @return Path
697    */
698   protected Path computeFilename(long filenum) {
699     if (filenum < 0) {
700       throw new RuntimeException("hlog file number can't be < 0");
701     }
702     return new Path(dir, prefix + "." + filenum);
703   }
704 
705   /**
706    * Shut down the log and delete the log directory
707    *
708    * @throws IOException
709    */
710   public void closeAndDelete() throws IOException {
711     close();
712     FileStatus[] files = fs.listStatus(this.dir);
713     for(FileStatus file : files) {
714       fs.rename(file.getPath(),
715           getHLogArchivePath(this.oldLogDir, file.getPath()));
716     }
717     LOG.debug("Moved " + files.length + " log files to " +
718         FSUtils.getPath(this.oldLogDir));
719     fs.delete(dir, true);
720   }
721 
722   /**
723    * Shut down the log.
724    *
725    * @throws IOException
726    */
727   public void close() throws IOException {
728     try {
729       logSyncerThread.interrupt();
730       // Make sure we synced everything
731       logSyncerThread.join(this.optionalFlushInterval*2);
732     } catch (InterruptedException e) {
733       LOG.error("Exception while waiting for syncer thread to die", e);
734     }
735 
736     cacheFlushLock.lock();
737     try {
738       synchronized (updateLock) {
739         this.closed = true;
740         if (LOG.isDebugEnabled()) {
741           LOG.debug("closing hlog writer in " + this.dir.toString());
742         }
743         this.writer.close();
744       }
745     } finally {
746       cacheFlushLock.unlock();
747     }
748   }
749 
750    /** Append an entry to the log.
751    *
752    * @param regionInfo
753    * @param logEdit
754    * @param now Time of this edit write.
755    * @throws IOException
756    */
757   public void append(HRegionInfo regionInfo, WALEdit logEdit,
758     final long now,
759     final boolean isMetaRegion)
760   throws IOException {
761     byte [] regionName = regionInfo.getRegionName();
762     byte [] tableName = regionInfo.getTableDesc().getName();
763     this.append(regionInfo, makeKey(regionName, tableName, -1, now), logEdit);
764   }
765 
766   /**
767    * @param now
768    * @param regionName
769    * @param tableName
770    * @return New log key.
771    */
772   protected HLogKey makeKey(byte[] regionName, byte[] tableName, long seqnum, long now) {
773     return new HLogKey(regionName, tableName, seqnum, now);
774   }
775 
776 
777 
778   /** Append an entry to the log.
779    *
780    * @param regionInfo
781    * @param logEdit
782    * @param logKey
783    * @throws IOException
784    */
785   public void append(HRegionInfo regionInfo, HLogKey logKey, WALEdit logEdit)
786   throws IOException {
787     if (this.closed) {
788       throw new IOException("Cannot append; log is closed");
789     }
790     byte [] regionName = regionInfo.getRegionName();
791     synchronized (updateLock) {
792       long seqNum = obtainSeqNum();
793       logKey.setLogSeqNum(seqNum);
794       // The 'lastSeqWritten' map holds the sequence number of the oldest
795       // write for each region (i.e. the first edit added to the particular
796       // memstore). When the cache is flushed, the entry for the
797       // region being flushed is removed if the sequence number of the flush
798       // is greater than or equal to the value in lastSeqWritten.
799       this.lastSeqWritten.putIfAbsent(regionName, Long.valueOf(seqNum));
800       doWrite(regionInfo, logKey, logEdit);
801       this.unflushedEntries.incrementAndGet();
802       this.numEntries.incrementAndGet();
803     }
804 
805     // sync txn to file system
806     this.sync(regionInfo.isMetaRegion());
807   }
808 
809   /**
810    * Append a set of edits to the log. Log edits are keyed by regionName,
811    * rowname, and log-sequence-id.
812    *
813    * Later, if we sort by these keys, we obtain all the relevant edits for a
814    * given key-range of the HRegion (TODO). Any edits that do not have a
815    * matching COMPLETE_CACHEFLUSH message can be discarded.
816    *
817    * <p>
818    * Logs cannot be restarted once closed, or once the HLog process dies. Each
819    * time the HLog starts, it must create a new log. This means that other
820    * systems should process the log appropriately upon each startup (and prior
821    * to initializing HLog).
822    *
823    * synchronized prevents appends during the completion of a cache flush or for
824    * the duration of a log roll.
825    *
826    * @param info
827    * @param tableName
828    * @param edits
829    * @param now
830    * @throws IOException
831    */
832   public void append(HRegionInfo info, byte [] tableName, WALEdit edits,
833     final long now)
834   throws IOException {
835     if (edits.isEmpty()) return;
836     
837     byte[] regionName = info.getRegionName();
838     if (this.closed) {
839       throw new IOException("Cannot append; log is closed");
840     }
841     synchronized (this.updateLock) {
842       long seqNum = obtainSeqNum();
843       // The 'lastSeqWritten' map holds the sequence number of the oldest
844       // write for each region (i.e. the first edit added to the particular
845       // memstore). . When the cache is flushed, the entry for the
846       // region being flushed is removed if the sequence number of the flush
847       // is greater than or equal to the value in lastSeqWritten.
848       this.lastSeqWritten.putIfAbsent(regionName, seqNum);
849       HLogKey logKey = makeKey(regionName, tableName, seqNum, now);
850       doWrite(info, logKey, edits);
851       this.numEntries.incrementAndGet();
852 
853       // Only count 1 row as an unflushed entry.
854       this.unflushedEntries.incrementAndGet();
855     }
856     // sync txn to file system
857     this.sync(info.isMetaRegion());
858   }
859 
860   /**
861    * This thread is responsible to call syncFs and buffer up the writers while
862    * it happens.
863    */
864    class LogSyncer extends Thread {
865 
866     // Using fairness to make sure locks are given in order
867     private final ReentrantLock lock = new ReentrantLock(true);
868 
869     // Condition used to wait until we have something to sync
870     private final Condition queueEmpty = lock.newCondition();
871 
872     // Condition used to signal that the sync is done
873     private final Condition syncDone = lock.newCondition();
874 
875     private final long optionalFlushInterval;
876 
877     private boolean syncerShuttingDown = false;
878 
879     LogSyncer(long optionalFlushInterval) {
880       this.optionalFlushInterval = optionalFlushInterval;
881     }
882 
883     @Override
884     public void run() {
885       try {
886         lock.lock();
887         // awaiting with a timeout doesn't always
888         // throw exceptions on interrupt
889         while(!this.isInterrupted()) {
890 
891           // Wait until something has to be hflushed or do it if we waited
892           // enough time (useful if something appends but does not hflush).
893           // 0 or less means that it timed out and maybe waited a bit more.
894           if (!(queueEmpty.awaitNanos(
895               this.optionalFlushInterval*1000000) <= 0)) {
896             forceSync = true;
897           }
898 
899           // We got the signal, let's hflush. We currently own the lock so new
900           // writes are waiting to acquire it in addToSyncQueue while the ones
901           // we hflush are waiting on await()
902           hflush();
903 
904           // Release all the clients waiting on the hflush. Notice that we still
905           // own the lock until we get back to await at which point all the
906           // other threads waiting will first acquire and release locks
907           syncDone.signalAll();
908         }
909       } catch (IOException e) {
910         LOG.error("Error while syncing, requesting close of hlog ", e);
911         requestLogRoll();
912       } catch (InterruptedException e) {
913         LOG.debug(getName() + "interrupted while waiting for sync requests");
914       } finally {
915         syncerShuttingDown = true;
916         syncDone.signalAll();
917         lock.unlock();
918         LOG.info(getName() + " exiting");
919       }
920     }
921 
922     /**
923      * This method first signals the thread that there's a sync needed
924      * and then waits for it to happen before returning.
925      */
926     public void addToSyncQueue(boolean force) {
927 
928       // Don't bother if somehow our append was already hflushed
929       if (unflushedEntries.get() == 0) {
930         return;
931       }
932       lock.lock();
933       try {
934         if (syncerShuttingDown) {
935           LOG.warn(getName() + " was shut down while waiting for sync");
936           return;
937         }
938         if(force) {
939           forceSync = true;
940         }
941         // Wake the thread
942         queueEmpty.signal();
943 
944         // Wait for it to hflush
945         syncDone.await();
946       } catch (InterruptedException e) {
947         LOG.debug(getName() + " was interrupted while waiting for sync", e);
948       }
949       finally {
950         lock.unlock();
951       }
952     }
953   }
954 
955   public void sync(){
956     sync(false);
957   }
958 
959   /**
960    * This method calls the LogSyncer in order to group commit the sync
961    * with other threads.
962    * @param force For catalog regions, force the sync to happen
963    */
964   public void sync(boolean force) {
965     logSyncerThread.addToSyncQueue(force);
966   }
967 
968   public void hflush() throws IOException {
969     synchronized (this.updateLock) {
970       if (this.closed) {
971         return;
972       }
973       boolean logRollRequested = false;
974       if (this.forceSync ||
975           this.unflushedEntries.get() >= this.flushlogentries) {
976         try {
977           long now = System.currentTimeMillis();
978           this.writer.sync();
979           syncTime += System.currentTimeMillis() - now;
980           syncOps++;
981           this.forceSync = false;
982           this.unflushedEntries.set(0);
983 
984           // if the number of replicas in HDFS has fallen below the initial
985           // value, then roll logs.
986           try {
987             int numCurrentReplicas = getLogReplication();
988             if (numCurrentReplicas != 0 &&
989                 numCurrentReplicas < this.initialReplication) {
990               LOG.warn("HDFS pipeline error detected. " +
991                   "Found " + numCurrentReplicas + " replicas but expecting " +
992                   this.initialReplication + " replicas. " +
993                   " Requesting close of hlog.");
994               requestLogRoll();
995               logRollRequested = true;
996             }
997           } catch (Exception e) {
998               LOG.warn("Unable to invoke DFSOutputStream.getNumCurrentReplicas" + e +
999                        " still proceeding ahead...");
1000           }
1001         } catch (IOException e) {
1002           LOG.fatal("Could not append. Requesting close of hlog", e);
1003           requestLogRoll();
1004           throw e;
1005         }
1006       }
1007 
1008       if (!logRollRequested && (this.writer.getLength() > this.logrollsize)) {
1009         requestLogRoll();
1010       }
1011     }
1012   }
1013 
1014   /**
1015    * This method gets the datanode replication count for the current HLog.
1016    *
1017    * If the pipeline isn't started yet or is empty, you will get the default
1018    * replication factor.  Therefore, if this function returns 0, it means you
1019    * are not properly running with the HDFS-826 patch.
1020    * @throws InvocationTargetException
1021    * @throws IllegalAccessException
1022    * @throws IllegalArgumentException
1023    *
1024    * @throws Exception
1025    */
1026   int getLogReplication() throws IllegalArgumentException, IllegalAccessException, InvocationTargetException {
1027     if(this.getNumCurrentReplicas != null && this.hdfs_out != null) {
1028       Object repl = this.getNumCurrentReplicas.invoke(this.hdfs_out, NO_ARGS);
1029       if (repl instanceof Integer) {
1030         return ((Integer)repl).intValue();
1031       }
1032     }
1033     return 0;
1034   }
1035 
1036   boolean canGetCurReplicas() {
1037     return this.getNumCurrentReplicas != null;
1038   }
1039 
1040   public void hsync() throws IOException {
1041     // Not yet implemented up in hdfs so just call hflush.
1042     hflush();
1043   }
1044 
1045   private void requestLogRoll() {
1046     if (this.listener != null) {
1047       this.listener.logRollRequested();
1048     }
1049   }
1050 
1051   protected void doWrite(HRegionInfo info, HLogKey logKey, WALEdit logEdit)
1052   throws IOException {
1053     if (!this.enabled) {
1054       return;
1055     }
1056     if (!this.logEntryVisitors.isEmpty()) {
1057       for (LogEntryVisitor visitor : this.logEntryVisitors) {
1058         visitor.visitLogEntryBeforeWrite(info, logKey, logEdit);
1059       }
1060     }
1061     try {
1062       long now = System.currentTimeMillis();
1063       this.writer.append(new HLog.Entry(logKey, logEdit));
1064       long took = System.currentTimeMillis() - now;
1065       writeTime += took;
1066       writeOps++;
1067       if (took > 1000) {
1068         LOG.warn(Thread.currentThread().getName() + " took " + took +
1069           "ms appending an edit to hlog; editcount=" + this.numEntries.get());
1070       }
1071     } catch (IOException e) {
1072       LOG.fatal("Could not append. Requesting close of hlog", e);
1073       requestLogRoll();
1074       throw e;
1075     }
1076   }
1077 
1078   /** @return How many items have been added to the log */
1079   int getNumEntries() {
1080     return numEntries.get();
1081   }
1082 
1083   /**
1084    * Obtain a log sequence number.
1085    */
1086   private long obtainSeqNum() {
1087     return this.logSeqNum.incrementAndGet();
1088   }
1089 
1090   /** @return the number of log files in use */
1091   int getNumLogFiles() {
1092     return outputfiles.size();
1093   }
1094 
1095   /**
1096    * By acquiring a log sequence ID, we can allow log messages to continue while
1097    * we flush the cache.
1098    *
1099    * Acquire a lock so that we do not roll the log between the start and
1100    * completion of a cache-flush. Otherwise the log-seq-id for the flush will
1101    * not appear in the correct logfile.
1102    *
1103    * @return sequence ID to pass {@link #completeCacheFlush(byte[], byte[], long, boolean)}
1104    * (byte[], byte[], long)}
1105    * @see #completeCacheFlush(byte[], byte[], long, boolean)
1106    * @see #abortCacheFlush()
1107    */
1108   public long startCacheFlush() {
1109     this.cacheFlushLock.lock();
1110     return obtainSeqNum();
1111   }
1112 
1113   /**
1114    * Complete the cache flush
1115    *
1116    * Protected by cacheFlushLock
1117    *
1118    * @param regionName
1119    * @param tableName
1120    * @param logSeqId
1121    * @throws IOException
1122    */
1123   public void completeCacheFlush(final byte [] regionName, final byte [] tableName,
1124     final long logSeqId,
1125     final boolean isMetaRegion)
1126   throws IOException {
1127     try {
1128       if (this.closed) {
1129         return;
1130       }
1131       synchronized (updateLock) {
1132         long now = System.currentTimeMillis();
1133         WALEdit edit = completeCacheFlushLogEdit();
1134         HLogKey key = makeKey(regionName, tableName, logSeqId,
1135             System.currentTimeMillis());
1136         this.writer.append(new Entry(key, edit));
1137         writeTime += System.currentTimeMillis() - now;
1138         writeOps++;
1139         this.numEntries.incrementAndGet();
1140         Long seq = this.lastSeqWritten.get(regionName);
1141         if (seq != null && logSeqId >= seq.longValue()) {
1142           this.lastSeqWritten.remove(regionName);
1143         }
1144       }
1145       // sync txn to file system
1146       this.sync(isMetaRegion);
1147 
1148     } finally {
1149       this.cacheFlushLock.unlock();
1150     }
1151   }
1152 
1153   private WALEdit completeCacheFlushLogEdit() {
1154     KeyValue kv = new KeyValue(METAROW, METAFAMILY, null,
1155       System.currentTimeMillis(), COMPLETE_CACHE_FLUSH);
1156     WALEdit e = new WALEdit();
1157     e.add(kv);
1158     return e;
1159   }
1160 
1161   /**
1162    * Abort a cache flush.
1163    * Call if the flush fails. Note that the only recovery for an aborted flush
1164    * currently is a restart of the regionserver so the snapshot content dropped
1165    * by the failure gets restored to the memstore.
1166    */
1167   public void abortCacheFlush() {
1168     this.cacheFlushLock.unlock();
1169   }
1170 
1171   /**
1172    * @param family
1173    * @return true if the column is a meta column
1174    */
1175   public static boolean isMetaFamily(byte [] family) {
1176     return Bytes.equals(METAFAMILY, family);
1177   }
1178 
1179   /**
1180    * Split up a bunch of regionserver commit log files that are no longer
1181    * being written to, into new files, one per region for region to replay on
1182    * startup. Delete the old log files when finished.
1183    *
1184    * @param rootDir qualified root directory of the HBase instance
1185    * @param srcDir Directory of log files to split: e.g.
1186    *                <code>${ROOTDIR}/log_HOST_PORT</code>
1187    * @param oldLogDir directory where processed (split) logs will be archived to
1188    * @param fs FileSystem
1189    * @param conf Configuration
1190    * @throws IOException will throw if corrupted hlogs aren't tolerated
1191    * @return the list of splits
1192    */
1193   public static List<Path> splitLog(final Path rootDir, final Path srcDir,
1194     Path oldLogDir, final FileSystem fs, final Configuration conf)
1195   throws IOException {
1196 
1197     long millis = System.currentTimeMillis();
1198     List<Path> splits = null;
1199     if (!fs.exists(srcDir)) {
1200       // Nothing to do
1201       return splits;
1202     }
1203     FileStatus [] logfiles = fs.listStatus(srcDir);
1204     if (logfiles == null || logfiles.length == 0) {
1205       // Nothing to do
1206       return splits;
1207     }
1208     LOG.info("Splitting " + logfiles.length + " hlog(s) in " +
1209       srcDir.toString());
1210     splits = splitLog(rootDir, srcDir, oldLogDir, logfiles, fs, conf);
1211     try {
1212       FileStatus[] files = fs.listStatus(srcDir);
1213       for(FileStatus file : files) {
1214         Path newPath = getHLogArchivePath(oldLogDir, file.getPath());
1215         LOG.info("Moving " +  FSUtils.getPath(file.getPath()) + " to " +
1216                    FSUtils.getPath(newPath));
1217         fs.rename(file.getPath(), newPath);
1218       }
1219       LOG.debug("Moved " + files.length + " log files to " +
1220         FSUtils.getPath(oldLogDir));
1221       fs.delete(srcDir, true);
1222     } catch (IOException e) {
1223       e = RemoteExceptionHandler.checkIOException(e);
1224       IOException io = new IOException("Cannot delete: " + srcDir);
1225       io.initCause(e);
1226       throw io;
1227     }
1228     long endMillis = System.currentTimeMillis();
1229     LOG.info("hlog file splitting completed in " + (endMillis - millis) +
1230         " millis for " + srcDir.toString());
1231     return splits;
1232   }
1233 
1234   // Private immutable datastructure to hold Writer and its Path.
1235   private final static class WriterAndPath {
1236     final Path p;
1237     final Writer w;
1238     WriterAndPath(final Path p, final Writer w) {
1239       this.p = p;
1240       this.w = w;
1241     }
1242   }
1243 
1244   @SuppressWarnings("unchecked")
1245   public static Class<? extends HLogKey> getKeyClass(Configuration conf) {
1246      return (Class<? extends HLogKey>)
1247        conf.getClass("hbase.regionserver.hlog.keyclass", HLogKey.class);
1248   }
1249 
1250   public static HLogKey newKey(Configuration conf) throws IOException {
1251     Class<? extends HLogKey> keyClass = getKeyClass(conf);
1252     try {
1253       return keyClass.newInstance();
1254     } catch (InstantiationException e) {
1255       throw new IOException("cannot create hlog key");
1256     } catch (IllegalAccessException e) {
1257       throw new IOException("cannot create hlog key");
1258     }
1259   }
1260 
1261   /**
1262    * Sorts the HLog edits in the given list of logfiles (that are a mix of edits on multiple regions)
1263    * by region and then splits them per region directories, in batches of (hbase.hlog.split.batch.size)
1264    *
1265    * A batch consists of a set of log files that will be sorted in a single map of edits indexed by region
1266    * the resulting map will be concurrently written by multiple threads to their corresponding regions
1267    *
1268    * Each batch consists of more more log files that are
1269    *  - recovered (files is opened for append then closed to ensure no process is writing into it)
1270    *  - parsed (each edit in the log is appended to a list of edits indexed by region
1271    *    see {@link #parseHLog} for more details)
1272    *  - marked as either processed or corrupt depending on parsing outcome
1273    *  - the resulting edits indexed by region are concurrently written to their corresponding region
1274    *    region directories
1275    *  - original files are then archived to a different directory
1276    *
1277    *
1278    *
1279    * @param rootDir  hbase directory
1280    * @param srcDir   logs directory
1281    * @param oldLogDir directory where processed logs are archived to
1282    * @param logfiles the list of log files to split
1283    * @param fs
1284    * @param conf
1285    * @return
1286    * @throws IOException
1287    */
1288   private static List<Path> splitLog(final Path rootDir, final Path srcDir,
1289     Path oldLogDir, final FileStatus[] logfiles, final FileSystem fs,
1290     final Configuration conf)
1291   throws IOException {
1292     List<Path> processedLogs = new ArrayList<Path>();
1293     List<Path> corruptedLogs = new ArrayList<Path>();
1294     final Map<byte [], WriterAndPath> logWriters =
1295       Collections.synchronizedMap(
1296         new TreeMap<byte [], WriterAndPath>(Bytes.BYTES_COMPARATOR));
1297     List<Path> splits = null;
1298 
1299     // Number of logs in a read batch
1300     // More means faster but bigger mem consumption
1301     //TODO make a note on the conf rename and update hbase-site.xml if needed
1302     int logFilesPerStep = conf.getInt("hbase.hlog.split.batch.size", 3);
1303      boolean skipErrors = conf.getBoolean("hbase.hlog.split.skip.errors", false);
1304 
1305 
1306     try {
1307       int i = -1;
1308       while (i < logfiles.length) {
1309         final Map<byte[], LinkedList<Entry>> editsByRegion =
1310           new TreeMap<byte[], LinkedList<Entry>>(Bytes.BYTES_COMPARATOR);
1311         for (int j = 0; j < logFilesPerStep; j++) {
1312           i++;
1313           if (i == logfiles.length) {
1314             break;
1315           }
1316           FileStatus log = logfiles[i];
1317           Path logPath = log.getPath();
1318           long logLength = log.getLen();
1319           LOG.debug("Splitting hlog " + (i + 1) + " of " + logfiles.length +
1320             ": " + logPath + ", length=" + logLength );
1321           try {
1322             recoverFileLease(fs, logPath, conf);
1323             parseHLog(log, editsByRegion, fs, conf);
1324             processedLogs.add(logPath);
1325           } catch (EOFException eof) {
1326             // truncated files are expected if a RS crashes (see HBASE-2643)
1327             LOG.info("EOF from hlog " + logPath + ".  continuing");
1328             processedLogs.add(logPath);
1329           } catch (IOException e) {
1330              if (skipErrors) {
1331                LOG.warn("Got while parsing hlog " + logPath +
1332                  ". Marking as corrupted", e);
1333                corruptedLogs.add(logPath);
1334              } else {
1335                throw e;
1336              }
1337           }
1338         }
1339         writeEditsBatchToRegions(editsByRegion, logWriters, rootDir, fs, conf);
1340       }
1341       if (fs.listStatus(srcDir).length > processedLogs.size() + corruptedLogs.size()) {
1342         throw new IOException("Discovered orphan hlog after split. Maybe " +
1343           "HRegionServer was not dead when we started");
1344       }
1345       archiveLogs(corruptedLogs, processedLogs, oldLogDir, fs, conf);
1346     } finally {
1347       splits = new ArrayList<Path>(logWriters.size());
1348       for (WriterAndPath wap : logWriters.values()) {
1349         wap.w.close();
1350         splits.add(wap.p);
1351         LOG.debug("Closed " + wap.p);
1352       }
1353     }
1354     return splits;
1355   }
1356 
1357 
1358   /**
1359    * Utility class that lets us keep track of the edit with it's key
1360    * Only used when splitting logs
1361    */
1362   public static class Entry implements Writable {
1363     private WALEdit edit;
1364     private HLogKey key;
1365 
1366     public Entry() {
1367       edit = new WALEdit();
1368       key = new HLogKey();
1369     }
1370 
1371     /**
1372      * Constructor for both params
1373      * @param edit log's edit
1374      * @param key log's key
1375      */
1376     public Entry(HLogKey key, WALEdit edit) {
1377       super();
1378       this.key = key;
1379       this.edit = edit;
1380     }
1381     /**
1382      * Gets the edit
1383      * @return edit
1384      */
1385     public WALEdit getEdit() {
1386       return edit;
1387     }
1388     /**
1389      * Gets the key
1390      * @return key
1391      */
1392     public HLogKey getKey() {
1393       return key;
1394     }
1395 
1396     @Override
1397     public String toString() {
1398       return this.key + "=" + this.edit;
1399     }
1400 
1401     @Override
1402     public void write(DataOutput dataOutput) throws IOException {
1403       this.key.write(dataOutput);
1404       this.edit.write(dataOutput);
1405     }
1406 
1407     @Override
1408     public void readFields(DataInput dataInput) throws IOException {
1409       this.key.readFields(dataInput);
1410       this.edit.readFields(dataInput);
1411     }
1412   }
1413 
1414   /**
1415    * Construct the HLog directory name
1416    *
1417    * @param info HServerInfo for server
1418    * @return the HLog directory name
1419    */
1420   public static String getHLogDirectoryName(HServerInfo info) {
1421     return getHLogDirectoryName(info.getServerName());
1422   }
1423 
1424   /**
1425    * Construct the HLog directory name
1426    *
1427    * @param serverAddress
1428    * @param startCode
1429    * @return the HLog directory name
1430    */
1431   public static String getHLogDirectoryName(String serverAddress,
1432       long startCode) {
1433     if (serverAddress == null || serverAddress.length() == 0) {
1434       return null;
1435     }
1436     return getHLogDirectoryName(
1437         HServerInfo.getServerName(serverAddress, startCode));
1438   }
1439 
1440   /**
1441    * Construct the HLog directory name
1442    *
1443    * @param serverName
1444    * @return the HLog directory name
1445    */
1446   public static String getHLogDirectoryName(String serverName) {
1447     StringBuilder dirName = new StringBuilder(HConstants.HREGION_LOGDIR_NAME);
1448     dirName.append("/");
1449     dirName.append(serverName);
1450     return dirName.toString();
1451   }
1452 
1453   public static boolean validateHLogFilename(String filename) {
1454     return pattern.matcher(filename).matches();
1455   }
1456 
1457   private static Path getHLogArchivePath(Path oldLogDir, Path p) {
1458     return new Path(oldLogDir, p.getName());
1459   }
1460 
1461   /**
1462    * Takes splitLogsMap and concurrently writes them to region directories using a thread pool
1463    *
1464    * @param splitLogsMap map that contains the log splitting result indexed by region
1465    * @param logWriters map that contains a writer per region
1466    * @param rootDir hbase root dir
1467    * @param fs
1468    * @param conf
1469    * @throws IOException
1470    */
1471   private static void writeEditsBatchToRegions(
1472     final Map<byte[], LinkedList<Entry>> splitLogsMap,
1473     final Map<byte[], WriterAndPath> logWriters,
1474     final Path rootDir, final FileSystem fs, final Configuration conf)
1475   throws IOException {
1476     // Number of threads to use when log splitting to rewrite the logs.
1477     // More means faster but bigger mem consumption.
1478     int logWriterThreads =
1479       conf.getInt("hbase.regionserver.hlog.splitlog.writer.threads", 3);
1480     boolean skipErrors = conf.getBoolean("hbase.skip.errors", false);
1481     HashMap<byte[], Future> writeFutureResult = new HashMap<byte[], Future>();
1482     NamingThreadFactory f  = new NamingThreadFactory(
1483             "SplitWriter-%1$d", Executors.defaultThreadFactory());
1484     ThreadPoolExecutor threadPool = (ThreadPoolExecutor)Executors.newFixedThreadPool(logWriterThreads, f);
1485     for (final byte [] region : splitLogsMap.keySet()) {
1486       Callable splitter = createNewSplitter(rootDir, logWriters, splitLogsMap, region, fs, conf);
1487       writeFutureResult.put(region, threadPool.submit(splitter));
1488     }
1489 
1490     threadPool.shutdown();
1491     // Wait for all threads to terminate
1492     try {
1493       for (int j = 0; !threadPool.awaitTermination(5, TimeUnit.SECONDS); j++) {
1494         String message = "Waiting for hlog writers to terminate, elapsed " + j * 5 + " seconds";
1495         if (j < 30) {
1496           LOG.debug(message);
1497         } else {
1498           LOG.info(message);
1499         }
1500 
1501       }
1502     } catch(InterruptedException ex) {
1503       LOG.warn("Hlog writers were interrupted, possible data loss!");
1504       if (!skipErrors) {
1505         throw new IOException("Could not finish writing log entries",  ex);
1506         //TODO  maybe we should fail here regardless if skipErrors is active or not
1507       }
1508     }
1509 
1510     for (Map.Entry<byte[], Future> entry : writeFutureResult.entrySet()) {
1511       try {
1512         entry.getValue().get();
1513       } catch (ExecutionException e) {
1514         throw (new IOException(e.getCause()));
1515       } catch (InterruptedException e1) {
1516         LOG.warn("Writer for region " +  Bytes.toString(entry.getKey()) +
1517                 " was interrupted, however the write process should have " +
1518                 "finished. Throwing up ", e1);
1519         throw (new IOException(e1.getCause()));
1520       }
1521     }
1522   }
1523 
1524   /*
1525    * Parse a single hlog and put the edits in @splitLogsMap
1526    *
1527    * @param logfile to split
1528    * @param splitLogsMap output parameter: a map with region names as keys and a
1529    * list of edits as values
1530    * @param fs the filesystem
1531    * @param conf the configuration
1532    * @throws IOException if hlog is corrupted, or can't be open
1533    */
1534   private static void parseHLog(final FileStatus logfile,
1535     final Map<byte[], LinkedList<Entry>> splitLogsMap, final FileSystem fs,
1536     final Configuration conf)
1537   throws IOException {
1538     // Check for possibly empty file. With appends, currently Hadoop reports a
1539     // zero length even if the file has been sync'd. Revisit if HDFS-376 or
1540     // HDFS-878 is committed.
1541     long length = logfile.getLen();
1542     if (length <= 0) {
1543       LOG.warn("File " + logfile.getPath() + " might be still open, length is 0");
1544     }
1545     Path path = logfile.getPath();
1546     Reader in;
1547     int editsCount = 0;
1548     try {
1549       in = HLog.getReader(fs, path, conf);
1550     } catch (EOFException e) {
1551       if (length <= 0) {
1552         //TODO should we ignore an empty, not-last log file if skip.errors is false?
1553         //Either way, the caller should decide what to do. E.g. ignore if this is the last
1554         //log in sequence.
1555         //TODO is this scenario still possible if the log has been recovered (i.e. closed)
1556         LOG.warn("Could not open " + path + " for reading. File is empty" + e);
1557         return;
1558       } else {
1559         throw e;
1560       }
1561     }
1562     try {
1563       Entry entry;
1564       while ((entry = in.next()) != null) {
1565         byte[] region = entry.getKey().getRegionName();
1566         LinkedList<Entry> queue = splitLogsMap.get(region);
1567         if (queue == null) {
1568           queue = new LinkedList<Entry>();
1569           splitLogsMap.put(region, queue);
1570         }
1571         queue.addLast(entry);
1572         editsCount++;
1573       }
1574     } finally {
1575       LOG.debug("Pushed=" + editsCount + " entries from " + path);
1576       try {
1577         if (in != null) {
1578           in.close();
1579         }
1580       } catch (IOException e) {
1581         LOG.warn("Close log reader in finally threw exception -- continuing", e);
1582       }
1583     }
1584   }
1585 
1586   private static Callable<Void> createNewSplitter(final Path rootDir,
1587     final Map<byte[], WriterAndPath> logWriters,
1588     final Map<byte[], LinkedList<Entry>> logEntries,
1589     final byte[] region, final FileSystem fs, final Configuration conf) {
1590     return new Callable<Void>() {
1591       public String getName() {
1592         return "Split writer thread for region " + Bytes.toStringBinary(region);
1593       }
1594 
1595       @Override
1596       public Void call() throws IOException {
1597         LinkedList<Entry> entries = logEntries.get(region);
1598         LOG.debug(this.getName()+" got " + entries.size() + " to process");
1599         long threadTime = System.currentTimeMillis();
1600         try {
1601           int editsCount = 0;
1602           WriterAndPath wap = logWriters.get(region);
1603           for (Entry logEntry: entries) {
1604             if (wap == null) {
1605               Path regionedits = getRegionSplitEditsPath(fs, logEntry, rootDir);
1606               if (fs.exists(regionedits)) {
1607                 LOG.warn("Found existing old edits file. It could be the " +
1608                   "result of a previous failed split attempt. Deleting " +
1609                   regionedits + ", length=" + fs.getFileStatus(regionedits).getLen());
1610                 if (!fs.delete(regionedits, false)) {
1611                   LOG.warn("Failed delete of old " + regionedits);
1612                 }
1613               }
1614               Writer w = createWriter(fs, regionedits, conf);
1615               wap = new WriterAndPath(regionedits, w);
1616               logWriters.put(region, wap);
1617               LOG.debug("Creating writer path=" + regionedits +
1618                 " region=" + Bytes.toStringBinary(region));
1619             }
1620             wap.w.append(logEntry);
1621             editsCount++;
1622           }
1623           LOG.debug(this.getName() + " Applied " + editsCount +
1624             " total edits to " + Bytes.toStringBinary(region) +
1625             " in " + (System.currentTimeMillis() - threadTime) + "ms");
1626         } catch (IOException e) {
1627           e = RemoteExceptionHandler.checkIOException(e);
1628           LOG.fatal(this.getName() + " Got while writing log entry to log", e);
1629           throw e;
1630         }
1631         return null;
1632       }
1633     };
1634   }
1635 
1636   /**
1637    * Moves processed logs to a oldLogDir after successful processing
1638    * Moves corrupted logs (any log that couldn't be successfully parsed
1639    * to corruptDir (.corrupt) for later investigation
1640    *
1641    * @param corruptedLogs
1642    * @param processedLogs
1643    * @param oldLogDir
1644    * @param fs
1645    * @param conf
1646    * @throws IOException
1647    */
1648   private static void archiveLogs(final List<Path> corruptedLogs,
1649     final List<Path> processedLogs, final Path oldLogDir,
1650     final FileSystem fs, final Configuration conf)
1651   throws IOException{
1652     final Path corruptDir = new Path(conf.get(HConstants.HBASE_DIR),
1653       conf.get("hbase.regionserver.hlog.splitlog.corrupt.dir", ".corrupt"));
1654 
1655     fs.mkdirs(corruptDir);
1656     fs.mkdirs(oldLogDir);
1657 
1658     for (Path corrupted: corruptedLogs) {
1659       Path p = new Path(corruptDir, corrupted.getName());
1660       LOG.info("Moving corrupted log " + corrupted + " to " + p);
1661       fs.rename(corrupted, p);
1662     }
1663 
1664     for (Path p: processedLogs) {
1665       Path newPath = getHLogArchivePath(oldLogDir, p);
1666       fs.rename(p, newPath);
1667       LOG.info("Archived processed log " + p + " to " + newPath);
1668     }
1669   }
1670 
1671   /*
1672    * Path to a file under RECOVERED_EDITS_DIR directory of the region found in
1673    * <code>logEntry</code> named for the sequenceid in the passed
1674    * <code>logEntry</code>: e.g. /hbase/some_table/2323432434/recovered.edits/2332.
1675    * This method also ensures existence of RECOVERED_EDITS_DIR under the region
1676    * creating it if necessary.
1677    * @param fs
1678    * @param logEntry
1679    * @param rootDir HBase root dir.
1680    * @return Path to file into which to dump split log edits.
1681    * @throws IOException
1682    */
1683   private static Path getRegionSplitEditsPath(final FileSystem fs,
1684       final Entry logEntry, final Path rootDir)
1685   throws IOException {
1686     Path tableDir = HTableDescriptor.getTableDir(rootDir,
1687       logEntry.getKey().getTablename());
1688     Path regiondir = HRegion.getRegionDir(tableDir,
1689       HRegionInfo.encodeRegionName(logEntry.getKey().getRegionName()));
1690     Path dir = getRegionDirRecoveredEditsDir(regiondir);
1691     if (!fs.exists(dir)) {
1692       if (!fs.mkdirs(dir)) LOG.warn("mkdir failed on " + dir);
1693     }
1694     return new Path(dir,
1695       formatRecoveredEditsFileName(logEntry.getKey().getLogSeqNum()));
1696    }
1697 
1698   static String formatRecoveredEditsFileName(final long seqid) {
1699     return String.format("%019d", seqid);
1700   }
1701 
1702 
1703   /**
1704    * Returns sorted set of edit files made by wal-log splitter.
1705    * @param fs
1706    * @param regiondir
1707    * @return Files in passed <code>regiondir</code> as a sorted set.
1708    * @throws IOException
1709    */
1710   public static NavigableSet<Path> getSplitEditFilesSorted(final FileSystem fs,
1711       final Path regiondir)
1712   throws IOException {
1713     Path editsdir = getRegionDirRecoveredEditsDir(regiondir);
1714     FileStatus [] files = fs.listStatus(editsdir, new PathFilter () {
1715       @Override
1716       public boolean accept(Path p) {
1717         boolean result = false;
1718         try {
1719           // Return files and only files that match the editfile names pattern.
1720           // There can be other files in this directory other than edit files.
1721           // In particular, on error, we'll move aside the bad edit file giving
1722           // it a timestamp suffix.  See moveAsideBadEditsFile.
1723           Matcher m = EDITFILES_NAME_PATTERN.matcher(p.getName());
1724           result = fs.isFile(p) && m.matches();
1725         } catch (IOException e) {
1726           LOG.warn("Failed isFile check on " + p);
1727         }
1728         return result;
1729       }
1730     });
1731     NavigableSet<Path> filesSorted = new TreeSet<Path>();
1732     if (files == null) return filesSorted;
1733     for (FileStatus status: files) {
1734       filesSorted.add(status.getPath());
1735     }
1736     return filesSorted;
1737   }
1738 
1739   /**
1740    * Move aside a bad edits file.
1741    * @param fs
1742    * @param edits Edits file to move aside.
1743    * @return The name of the moved aside file.
1744    * @throws IOException
1745    */
1746   public static Path moveAsideBadEditsFile(final FileSystem fs,
1747       final Path edits)
1748   throws IOException {
1749     Path moveAsideName = new Path(edits.getParent(), edits.getName() + "." +
1750       System.currentTimeMillis());
1751     if (!fs.rename(edits, moveAsideName)) {
1752       LOG.warn("Rename failed from " + edits + " to " + moveAsideName);
1753     }
1754     return moveAsideName;
1755   }
1756 
1757   /**
1758    * @param regiondir This regions directory in the filesystem.
1759    * @return The directory that holds recovered edits files for the region
1760    * <code>regiondir</code>
1761    */
1762   public static Path getRegionDirRecoveredEditsDir(final Path regiondir) {
1763     return new Path(regiondir, RECOVERED_EDITS_DIR);
1764   }
1765 
1766   /**
1767    *
1768    * @param visitor
1769    */
1770   public void addLogEntryVisitor(LogEntryVisitor visitor) {
1771     this.logEntryVisitors.add(visitor);
1772   }
1773 
1774   /**
1775    * 
1776    * @param visitor
1777    */
1778   public void removeLogEntryVisitor(LogEntryVisitor visitor) {
1779     this.logEntryVisitors.remove(visitor);
1780   }
1781 
1782 
1783   public void addLogActionsListerner(LogActionsListener list) {
1784     LOG.info("Adding a listener");
1785     this.actionListeners.add(list);
1786   }
1787 
1788   public boolean removeLogActionsListener(LogActionsListener list) {
1789     return this.actionListeners.remove(list);
1790   }
1791 
1792   public static final long FIXED_OVERHEAD = ClassSize.align(
1793     ClassSize.OBJECT + (5 * ClassSize.REFERENCE) +
1794     ClassSize.ATOMIC_INTEGER + Bytes.SIZEOF_INT + (3 * Bytes.SIZEOF_LONG));
1795 
1796   private static void usage() {
1797     System.err.println("Usage: HLog <ARGS>");
1798     System.err.println("Arguments:");
1799     System.err.println(" --dump  Dump textual representation of passed one or more files");
1800     System.err.println("         For example: HLog --dump hdfs://example.com:9000/hbase/.logs/MACHINE/LOGFILE");
1801     System.err.println(" --split Split the passed directory of WAL logs");
1802     System.err.println("         For example: HLog --split hdfs://example.com:9000/hbase/.logs/DIR");
1803   }
1804 
1805   private static void dump(final Configuration conf, final Path p)
1806   throws IOException {
1807     FileSystem fs = FileSystem.get(conf);
1808     if (!fs.exists(p)) {
1809       throw new FileNotFoundException(p.toString());
1810     }
1811     if (!fs.isFile(p)) {
1812       throw new IOException(p + " is not a file");
1813     }
1814     Reader log = getReader(fs, p, conf);
1815     try {
1816       int count = 0;
1817       HLog.Entry entry;
1818       while ((entry = log.next()) != null) {
1819         System.out.println("#" + count + ", pos=" + log.getPosition() + " " +
1820           entry.toString());
1821         count++;
1822       }
1823     } finally {
1824       log.close();
1825     }
1826   }
1827 
1828   private static void split(final Configuration conf, final Path p)
1829   throws IOException {
1830     FileSystem fs = FileSystem.get(conf);
1831     if (!fs.exists(p)) {
1832       throw new FileNotFoundException(p.toString());
1833     }
1834     final Path baseDir = new Path(conf.get(HConstants.HBASE_DIR));
1835     final Path oldLogDir = new Path(baseDir, HConstants.HREGION_OLDLOGDIR_NAME);
1836     if (!fs.getFileStatus(p).isDir()) {
1837       throw new IOException(p + " is not a directory");
1838     }
1839     splitLog(baseDir, p, oldLogDir, fs, conf);
1840   }
1841 
1842   /**
1843    * Pass one or more log file names and it will either dump out a text version
1844    * on <code>stdout</code> or split the specified log files.
1845    *
1846    * @param args
1847    * @throws IOException
1848    */
1849   public static void main(String[] args) throws IOException {
1850     if (args.length < 2) {
1851       usage();
1852       System.exit(-1);
1853     }
1854     boolean dump = true;
1855     if (args[0].compareTo("--dump") != 0) {
1856       if (args[0].compareTo("--split") == 0) {
1857         dump = false;
1858 
1859       } else {
1860         usage();
1861         System.exit(-1);
1862       }
1863     }
1864     Configuration conf = HBaseConfiguration.create();
1865     for (int i = 1; i < args.length; i++) {
1866       Path logPath = new Path(args[i]);
1867       try {
1868         if (dump) {
1869           dump(conf, logPath);
1870         } else {
1871           split(conf, logPath);
1872         }
1873       } catch (Throwable t) {
1874         t.printStackTrace(System.err);
1875         System.exit(-1);
1876       }
1877     }
1878   }
1879 }