View Javadoc

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