View Javadoc

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