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