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.EOFException;
23  import java.io.IOException;
24  import java.io.InterruptedIOException;
25  import java.lang.reflect.Constructor;
26  import java.lang.reflect.InvocationTargetException;
27  import java.text.ParseException;
28  import java.util.ArrayList;
29  import java.util.Collections;
30  import java.util.LinkedList;
31  import java.util.List;
32  import java.util.Map;
33  import java.util.Set;
34  import java.util.TreeMap;
35  import java.util.TreeSet;
36  import java.util.concurrent.atomic.AtomicReference;
37  
38  import org.apache.commons.logging.Log;
39  import org.apache.commons.logging.LogFactory;
40  import org.apache.hadoop.conf.Configuration;
41  import org.apache.hadoop.fs.FileStatus;
42  import org.apache.hadoop.fs.FileSystem;
43  import org.apache.hadoop.fs.Path;
44  import org.apache.hadoop.hbase.HConstants;
45  import org.apache.hadoop.hbase.HTableDescriptor;
46  import org.apache.hadoop.hbase.RemoteExceptionHandler;
47  import org.apache.hadoop.hbase.io.HeapSize;
48  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
49  import org.apache.hadoop.hbase.monitoring.TaskMonitor;
50  import org.apache.hadoop.hbase.regionserver.HRegion;
51  import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry;
52  import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
53  import org.apache.hadoop.hbase.regionserver.wal.HLog.Writer;
54  import org.apache.hadoop.hbase.util.Bytes;
55  import org.apache.hadoop.hbase.util.CancelableProgressable;
56  import org.apache.hadoop.hbase.util.ClassSize;
57  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
58  import org.apache.hadoop.hbase.util.FSUtils;
59  import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
60  import org.apache.hadoop.io.MultipleIOException;
61  
62  import com.google.common.base.Preconditions;
63  import com.google.common.collect.Lists;
64  
65  /**
66   * This class is responsible for splitting up a bunch of regionserver commit log
67   * files that are no longer being written to, into new files, one per region for
68   * region to replay on startup. Delete the old log files when finished.
69   */
70  public class HLogSplitter {
71    private static final String LOG_SPLITTER_IMPL = "hbase.hlog.splitter.impl";
72  
73    /**
74     * Name of file that holds recovered edits written by the wal log splitting
75     * code, one per region
76     */
77    public static final String RECOVERED_EDITS = "recovered.edits";
78  
79  
80    static final Log LOG = LogFactory.getLog(HLogSplitter.class);
81  
82    private boolean hasSplit = false;
83    private long splitTime = 0;
84    private long splitSize = 0;
85  
86  
87    // Parameters for split process
88    protected final Path rootDir;
89    protected final Path srcDir;
90    protected final Path oldLogDir;
91    protected final FileSystem fs;
92    protected final Configuration conf;
93  
94    // Major subcomponents of the split process.
95    // These are separated into inner classes to make testing easier.
96    OutputSink outputSink;
97    EntryBuffers entryBuffers;
98  
99    // If an exception is thrown by one of the other threads, it will be
100   // stored here.
101   protected AtomicReference<Throwable> thrown = new AtomicReference<Throwable>();
102 
103   // Wait/notify for when data has been produced by the reader thread,
104   // consumed by the reader thread, or an exception occurred
105   Object dataAvailable = new Object();
106   
107   private MonitoredTask status;
108 
109 
110   /**
111    * Create a new HLogSplitter using the given {@link Configuration} and the
112    * <code>hbase.hlog.splitter.impl</code> property to derived the instance
113    * class to use.
114    * <p>
115    * @param conf
116    * @param rootDir hbase directory
117    * @param srcDir logs directory
118    * @param oldLogDir directory where processed logs are archived to
119    * @param fs FileSystem
120    * @return New HLogSplitter instance
121    */
122   public static HLogSplitter createLogSplitter(Configuration conf,
123       final Path rootDir, final Path srcDir,
124       Path oldLogDir, final FileSystem fs)  {
125 
126     @SuppressWarnings("unchecked")
127     Class<? extends HLogSplitter> splitterClass = (Class<? extends HLogSplitter>) conf
128         .getClass(LOG_SPLITTER_IMPL, HLogSplitter.class);
129     try {
130        Constructor<? extends HLogSplitter> constructor =
131          splitterClass.getConstructor(
132           Configuration.class, // conf
133           Path.class, // rootDir
134           Path.class, // srcDir
135           Path.class, // oldLogDir
136           FileSystem.class); // fs
137       return constructor.newInstance(conf, rootDir, srcDir, oldLogDir, fs);
138     } catch (IllegalArgumentException e) {
139       throw new RuntimeException(e);
140     } catch (InstantiationException e) {
141       throw new RuntimeException(e);
142     } catch (IllegalAccessException e) {
143       throw new RuntimeException(e);
144     } catch (InvocationTargetException e) {
145       throw new RuntimeException(e);
146     } catch (SecurityException e) {
147       throw new RuntimeException(e);
148     } catch (NoSuchMethodException e) {
149       throw new RuntimeException(e);
150     }
151   }
152 
153   public HLogSplitter(Configuration conf, Path rootDir, Path srcDir,
154       Path oldLogDir, FileSystem fs) {
155     this.conf = conf;
156     this.rootDir = rootDir;
157     this.srcDir = srcDir;
158     this.oldLogDir = oldLogDir;
159     this.fs = fs;
160 
161     entryBuffers = new EntryBuffers(
162         conf.getInt("hbase.regionserver.hlog.splitlog.buffersize",
163             128*1024*1024));
164     outputSink = new OutputSink();
165   }
166 
167   /**
168    * Split up a bunch of regionserver commit log files that are no longer being
169    * written to, into new files, one per region for region to replay on startup.
170    * Delete the old log files when finished.
171    *
172    * @throws IOException will throw if corrupted hlogs aren't tolerated
173    * @return the list of splits
174    */
175   public List<Path> splitLog()
176       throws IOException {
177     Preconditions.checkState(!hasSplit,
178         "An HLogSplitter instance may only be used once");
179     hasSplit = true;
180 
181     status = TaskMonitor.get().createStatus(
182         "Splitting logs in " + srcDir);
183     
184     long startTime = EnvironmentEdgeManager.currentTimeMillis();
185     
186     status.setStatus("Determining files to split...");
187     List<Path> splits = null;
188     if (!fs.exists(srcDir)) {
189       // Nothing to do
190       status.markComplete("No log directory existed to split.");
191       return splits;
192     }
193     FileStatus[] logfiles = fs.listStatus(srcDir);
194     if (logfiles == null || logfiles.length == 0) {
195       // Nothing to do
196       return splits;
197     }
198     logAndReport("Splitting " + logfiles.length + " hlog(s) in "
199     + srcDir.toString());
200     splits = splitLog(logfiles);
201 
202     splitTime = EnvironmentEdgeManager.currentTimeMillis() - startTime;
203     String msg = "hlog file splitting completed in " + splitTime +
204         " ms for " + srcDir.toString();
205     status.markComplete(msg);
206     LOG.info(msg);
207     return splits;
208   }
209   
210   private void logAndReport(String msg) {
211     status.setStatus(msg);
212     LOG.info(msg);
213   }
214 
215   /**
216    * @return time that this split took
217    */
218   public long getTime() {
219     return this.splitTime;
220   }
221 
222   /**
223    * @return aggregate size of hlogs that were split
224    */
225   public long getSize() {
226     return this.splitSize;
227   }
228 
229   /**
230    * @return a map from encoded region ID to the number of edits written out
231    * for that region.
232    */
233   Map<byte[], Long> getOutputCounts() {
234     Preconditions.checkState(hasSplit);
235     return outputSink.getOutputCounts();
236   }
237 
238   /**
239    * Splits the HLog edits in the given list of logfiles (that are a mix of edits
240    * on multiple regions) by region and then splits them per region directories,
241    * in batches of (hbase.hlog.split.batch.size)
242    * <p>
243    * This process is split into multiple threads. In the main thread, we loop
244    * through the logs to be split. For each log, we:
245    * <ul>
246    *   <li> Recover it (take and drop HDFS lease) to ensure no other process can write</li>
247    *   <li> Read each edit (see {@link #parseHLog}</li>
248    *   <li> Mark as "processed" or "corrupt" depending on outcome</li>
249    * </ul>
250    * <p>
251    * Each edit is passed into the EntryBuffers instance, which takes care of
252    * memory accounting and splitting the edits by region.
253    * <p>
254    * The OutputSink object then manages N other WriterThreads which pull chunks
255    * of edits from EntryBuffers and write them to the output region directories.
256    * <p>
257    * After the process is complete, the log files are archived to a separate
258    * directory.
259    */
260   private List<Path> splitLog(final FileStatus[] logfiles) throws IOException {
261     List<Path> processedLogs = new ArrayList<Path>();
262     List<Path> corruptedLogs = new ArrayList<Path>();
263     List<Path> splits = null;
264 
265     boolean skipErrors = conf.getBoolean("hbase.hlog.split.skip.errors", true);
266 
267     countTotalBytes(logfiles);
268     splitSize = 0;
269 
270     outputSink.startWriterThreads(entryBuffers);
271 
272     try {
273       int i = 0;
274       for (FileStatus log : logfiles) {
275        Path logPath = log.getPath();
276         long logLength = log.getLen();
277         splitSize += logLength;
278         logAndReport("Splitting hlog " + (i++ + 1) + " of " + logfiles.length
279             + ": " + logPath + ", length=" + logLength);
280         Reader in;
281         try {
282           in = getReader(fs, log, conf, skipErrors);
283           if (in != null) {
284             parseHLog(in, logPath, entryBuffers, fs, conf, skipErrors);
285             try {
286               in.close();
287             } catch (IOException e) {
288               LOG.warn("Close log reader threw exception -- continuing",
289                   e);
290             }
291           }
292           processedLogs.add(logPath);
293         } catch (CorruptedLogFileException e) {
294           LOG.info("Got while parsing hlog " + logPath +
295               ". Marking as corrupted", e);
296           corruptedLogs.add(logPath);
297           continue;
298         }
299       }
300       status.setStatus("Log splits complete. Checking for orphaned logs.");
301       
302       if (fs.listStatus(srcDir).length > processedLogs.size()
303           + corruptedLogs.size()) {
304         throw new OrphanHLogAfterSplitException(
305             "Discovered orphan hlog after split. Maybe the "
306             + "HRegionServer was not dead when we started");
307       }
308     } finally {
309       status.setStatus("Finishing writing output logs and closing down.");
310       splits = outputSink.finishWritingAndClose();
311     }
312     status.setStatus("Archiving logs after completed split");
313     archiveLogs(srcDir, corruptedLogs, processedLogs, oldLogDir, fs, conf);
314     return splits;
315   }
316 
317   /**
318    * @return the total size of the passed list of files.
319    */
320   private static long countTotalBytes(FileStatus[] logfiles) {
321     long ret = 0;
322     for (FileStatus stat : logfiles) {
323       ret += stat.getLen();
324     }
325     return ret;
326   }
327 
328   /**
329    * Splits a HLog file into region's recovered-edits directory
330    * <p>
331    * If the log file has N regions then N recovered.edits files will be
332    * produced. There is no buffering in this code. Instead it relies on the
333    * buffering in the SequenceFileWriter.
334    * <p>
335    * @param rootDir
336    * @param logfile
337    * @param fs
338    * @param conf
339    * @param reporter
340    * @return false if it is interrupted by the progress-able.
341    * @throws IOException
342    */
343   static public boolean splitLogFile(Path rootDir, FileStatus logfile,
344       FileSystem fs, Configuration conf, CancelableProgressable reporter)
345       throws IOException {
346     HLogSplitter s = new HLogSplitter(conf, rootDir, null, null /* oldLogDir */,
347         fs);
348     return s.splitLogFile(logfile, reporter);
349   }
350 
351   public boolean splitLogFile(FileStatus logfile,
352       CancelableProgressable reporter) throws IOException {
353     final Map<byte[], Object> logWriters = Collections.
354     synchronizedMap(new TreeMap<byte[], Object>(Bytes.BYTES_COMPARATOR));
355     boolean isCorrupted = false;
356     
357     Preconditions.checkState(status == null);
358     status = TaskMonitor.get().createStatus(
359         "Splitting log file " + logfile.getPath() +
360         "into a temporary staging area.");
361 
362     Object BAD_WRITER = new Object();
363 
364     boolean progress_failed = false;
365 
366     boolean skipErrors = conf.getBoolean("hbase.hlog.split.skip.errors",
367         HLog.SPLIT_SKIP_ERRORS_DEFAULT);
368     int interval = conf.getInt("hbase.splitlog.report.interval.loglines", 1024);
369     // How often to send a progress report (default 1/2 the zookeeper session
370     // timeout of if that not set, the split log DEFAULT_TIMEOUT)
371     int period = conf.getInt("hbase.splitlog.report.period",
372       conf.getInt("hbase.splitlog.manager.timeout", ZKSplitLog.DEFAULT_TIMEOUT) / 2);
373     int numOpenedFilesBeforeReporting =
374       conf.getInt("hbase.splitlog.report.openedfiles", 3);
375     Path logPath = logfile.getPath();
376     long logLength = logfile.getLen();
377     LOG.info("Splitting hlog: " + logPath + ", length=" + logLength);
378     status.setStatus("Opening log file");
379     Reader in = null;
380     try {
381       in = getReader(fs, logfile, conf, skipErrors);
382     } catch (CorruptedLogFileException e) {
383       LOG.warn("Could not get reader, corrupted log file " + logPath, e);
384       ZKSplitLog.markCorrupted(rootDir, logfile.getPath().getName(), fs);
385       isCorrupted = true;
386     }
387     if (in == null) {
388       status.markComplete("Was nothing to split in log file");
389       LOG.warn("Nothing to split in log file " + logPath);
390       return true;
391     }
392     long t = EnvironmentEdgeManager.currentTimeMillis();
393     long last_report_at = t;
394     if (reporter != null && reporter.progress() == false) {
395       status.markComplete("Failed: reporter.progress asked us to terminate");
396       return false;
397     }
398     // Report progress every so many edits and/or files opened (opening a file
399     // takes a bit of time).
400     int editsCount = 0;
401     int numNewlyOpenedFiles = 0;
402     Entry entry;
403     try {
404       while ((entry = getNextLogLine(in,logPath, skipErrors)) != null) {
405         byte[] region = entry.getKey().getEncodedRegionName();
406         Object o = logWriters.get(region);
407         if (o == BAD_WRITER) {
408           continue;
409         }
410         WriterAndPath wap = (WriterAndPath)o;
411         if (wap == null) {
412           wap = createWAP(region, entry, rootDir, fs, conf);
413           numNewlyOpenedFiles++;
414           if (wap == null) {
415             // ignore edits from this region. It doesn't exist anymore.
416             // It was probably already split.
417             logWriters.put(region, BAD_WRITER);
418             continue;
419           } else {
420             logWriters.put(region, wap);
421           }
422         }
423         wap.w.append(entry);
424         outputSink.updateRegionMaximumEditLogSeqNum(entry);
425         editsCount++;
426         // If sufficient edits have passed OR we've opened a few files, check if
427         // we should report progress.
428         if (editsCount % interval == 0 ||
429             (numNewlyOpenedFiles > numOpenedFilesBeforeReporting)) {
430           // Zero out files counter each time we fall in here.
431           numNewlyOpenedFiles = 0;
432           String countsStr = "edits=" + editsCount + ", files=" + logWriters.size();
433           status.setStatus("Split " + countsStr);
434           long t1 = EnvironmentEdgeManager.currentTimeMillis();
435           if ((t1 - last_report_at) > period) {
436             last_report_at = t;
437             if (reporter != null && reporter.progress() == false) {
438               status.markComplete("Failed: reporter.progress asked us to terminate; " + countsStr);
439               progress_failed = true;
440               return false;
441             }
442           }
443         }
444       }
445     } catch (CorruptedLogFileException e) {
446       LOG.warn("Could not parse, corrupted log file " + logPath, e);
447       ZKSplitLog.markCorrupted(rootDir, logfile.getPath().getName(), fs);
448       isCorrupted = true;
449     } catch (IOException e) {
450       e = RemoteExceptionHandler.checkIOException(e);
451       throw e;
452     } finally {
453       boolean allWritersClosed = false;
454       try {
455         int n = 0;
456         for (Map.Entry<byte[], Object> logWritersEntry : logWriters.entrySet()) {
457           Object o = logWritersEntry.getValue();
458           long t1 = EnvironmentEdgeManager.currentTimeMillis();
459           if ((t1 - last_report_at) > period) {
460             last_report_at = t;
461             if ((progress_failed == false) && (reporter != null) && (reporter.progress() == false)) {
462               progress_failed = true;
463             }
464           }
465           if (o == BAD_WRITER) {
466             continue;
467           }
468           n++;
469           WriterAndPath wap = (WriterAndPath) o;
470           wap.writerClosed = true;
471           wap.w.close();
472           LOG.debug("Closed " + wap.p);
473           Path dst = getCompletedRecoveredEditsFilePath(wap.p,
474               outputSink.getRegionMaximumEditLogSeqNum(logWritersEntry.getKey()));
475           if (!dst.equals(wap.p) && fs.exists(dst)) {
476             LOG.warn("Found existing old edits file. It could be the "
477                 + "result of a previous failed split attempt. Deleting " + dst + ", length="
478                 + fs.getFileStatus(dst).getLen());
479             if (!fs.delete(dst, false)) {
480               LOG.warn("Failed deleting of old " + dst);
481               throw new IOException("Failed deleting of old " + dst);
482             }
483           }
484           // Skip the unit tests which create a splitter that reads and writes the
485           // data without touching disk. TestHLogSplit#testThreading is an
486           // example.
487           if (fs.exists(wap.p)) {
488             if (!fs.rename(wap.p, dst)) {
489               throw new IOException("Failed renaming " + wap.p + " to " + dst);
490             }
491             LOG.debug("Rename " + wap.p + " to " + dst);
492           }
493         }
494         allWritersClosed = true;
495         String msg = "Processed " + editsCount + " edits across " + n + " regions"
496             + " threw away edits for " + (logWriters.size() - n) + " regions" + "; log file="
497             + logPath + " is corrupted = " + isCorrupted + " progress failed = " + progress_failed;
498         LOG.info(msg);
499         status.markComplete(msg);
500       } finally {
501         if (!allWritersClosed) {
502           for (Map.Entry<byte[], Object> logWritersEntry : logWriters.entrySet()) {
503             Object o = logWritersEntry.getValue();
504             if (o != BAD_WRITER) {
505               WriterAndPath wap = (WriterAndPath) o;
506               try {
507                 if (!wap.writerClosed) {
508                   wap.writerClosed = true;
509                   wap.w.close();
510                 }
511               } catch (IOException e) {
512                 LOG.debug("Exception while closing the writer :", e);
513               }
514             }
515           }
516         }
517         in.close();
518       }
519     }
520     return !progress_failed;
521   }
522 
523   /**
524    * Completes the work done by splitLogFile by archiving logs
525    * <p>
526    * It is invoked by SplitLogManager once it knows that one of the
527    * SplitLogWorkers have completed the splitLogFile() part. If the master
528    * crashes then this function might get called multiple times.
529    * <p>
530    * @param logfile
531    * @param conf
532    * @throws IOException
533    */
534   public static void finishSplitLogFile(String logfile, Configuration conf)
535       throws IOException {
536     Path rootdir = FSUtils.getRootDir(conf);
537     Path oldLogDir = new Path(rootdir, HConstants.HREGION_OLDLOGDIR_NAME);
538     finishSplitLogFile(rootdir, oldLogDir, logfile, conf);
539   }
540 
541   public static void finishSplitLogFile(Path rootdir, Path oldLogDir,
542       String logfile, Configuration conf) throws IOException {
543     List<Path> processedLogs = new ArrayList<Path>();
544     List<Path> corruptedLogs = new ArrayList<Path>();
545     FileSystem fs;
546     fs = rootdir.getFileSystem(conf);
547     Path logPath = new Path(logfile);
548     if (ZKSplitLog.isCorrupted(rootdir, logPath.getName(), fs)) {
549       corruptedLogs.add(logPath);
550     } else {
551       processedLogs.add(logPath);
552     }
553     archiveLogs(null, corruptedLogs, processedLogs, oldLogDir, fs, conf);
554     Path stagingDir = ZKSplitLog.getSplitLogDir(rootdir, logPath.getName());
555     fs.delete(stagingDir, true);
556   }
557 
558   /**
559    * Moves processed logs to a oldLogDir after successful processing Moves
560    * corrupted logs (any log that couldn't be successfully parsed to corruptDir
561    * (.corrupt) for later investigation
562    *
563    * @param corruptedLogs
564    * @param processedLogs
565    * @param oldLogDir
566    * @param fs
567    * @param conf
568    * @throws IOException
569    */
570   private static void archiveLogs(
571       final Path srcDir,
572       final List<Path> corruptedLogs,
573       final List<Path> processedLogs, final Path oldLogDir,
574       final FileSystem fs, final Configuration conf) throws IOException {
575     final Path corruptDir = new Path(conf.get(HConstants.HBASE_DIR), conf.get(
576         "hbase.regionserver.hlog.splitlog.corrupt.dir",  HConstants.CORRUPT_DIR_NAME));
577 
578     if (!fs.mkdirs(corruptDir)) {
579       LOG.info("Unable to mkdir " + corruptDir);
580     }
581     fs.mkdirs(oldLogDir);
582 
583     // this method can get restarted or called multiple times for archiving
584     // the same log files.
585     for (Path corrupted : corruptedLogs) {
586       Path p = new Path(corruptDir, corrupted.getName());
587       if (fs.exists(corrupted)) {
588         if (!fs.rename(corrupted, p)) {
589           LOG.warn("Unable to move corrupted log " + corrupted + " to " + p);
590         } else {
591           LOG.warn("Moving corrupted log " + corrupted + " to " + p);
592         }
593       }
594     }
595 
596     for (Path p : processedLogs) {
597       Path newPath = HLog.getHLogArchivePath(oldLogDir, p);
598       if (fs.exists(p)) {
599         if (!fs.rename(p, newPath)) {
600           LOG.warn("Unable to move  " + p + " to " + newPath);
601         } else {
602           LOG.debug("Archived processed log " + p + " to " + newPath);
603         }
604       }
605     }
606 
607     // distributed log splitting removes the srcDir (region's log dir) later
608     // when all the log files in that srcDir have been successfully processed
609     if (srcDir != null && !fs.delete(srcDir, true)) {
610       throw new IOException("Unable to delete src dir: " + srcDir);
611     }
612   }
613 
614   /**
615    * Path to a file under RECOVERED_EDITS_DIR directory of the region found in
616    * <code>logEntry</code> named for the sequenceid in the passed
617    * <code>logEntry</code>: e.g. /hbase/some_table/2323432434/recovered.edits/2332.
618    * This method also ensures existence of RECOVERED_EDITS_DIR under the region
619    * creating it if necessary.
620    * @param fs
621    * @param logEntry
622    * @param rootDir HBase root dir.
623    * @return Path to file into which to dump split log edits.
624    * @throws IOException
625    */
626   static Path getRegionSplitEditsPath(final FileSystem fs,
627       final Entry logEntry, final Path rootDir, boolean isCreate)
628   throws IOException {
629     Path tableDir = HTableDescriptor.getTableDir(rootDir, logEntry.getKey()
630         .getTablename());
631     Path regiondir = HRegion.getRegionDir(tableDir,
632         Bytes.toString(logEntry.getKey().getEncodedRegionName()));
633     Path dir = HLog.getRegionDirRecoveredEditsDir(regiondir);
634 
635     if (!fs.exists(regiondir)) {
636       LOG.info("This region's directory doesn't exist: "
637           + regiondir.toString() + ". It is very likely that it was" +
638           " already split so it's safe to discard those edits.");
639       return null;
640     }
641     if (isCreate && !fs.exists(dir)) {
642       if (!fs.mkdirs(dir)) LOG.warn("mkdir failed on " + dir);
643     }
644     // Append file name ends with RECOVERED_LOG_TMPFILE_SUFFIX to ensure
645     // region's replayRecoveredEdits will not delete it
646     String fileName = formatRecoveredEditsFileName(logEntry.getKey()
647         .getLogSeqNum());
648     fileName = getTmpRecoveredEditsFileName(fileName);
649     return new Path(dir, fileName);
650   }
651 
652   static String getTmpRecoveredEditsFileName(String fileName) {
653     return fileName + HLog.RECOVERED_LOG_TMPFILE_SUFFIX;
654   }
655 
656   /**
657    * Get the completed recovered edits file path, renaming it to be by last edit
658    * in the file from its first edit. Then we could use the name to skip
659    * recovered edits when doing {@link HRegion#replayRecoveredEditsIfAny}.
660    * @param srcPath
661    * @param maximumEditLogSeqNum
662    * @return dstPath take file's last edit log seq num as the name
663    */
664   static Path getCompletedRecoveredEditsFilePath(Path srcPath,
665       Long maximumEditLogSeqNum) {
666     String fileName = formatRecoveredEditsFileName(maximumEditLogSeqNum);
667     return new Path(srcPath.getParent(), fileName);
668   }
669 
670   static String formatRecoveredEditsFileName(final long seqid) {
671     return String.format("%019d", seqid);
672   }
673 
674   /**
675    * Parse a single hlog and put the edits in entryBuffers
676    *
677    * @param in the hlog reader
678    * @param path the path of the log file
679    * @param entryBuffers the buffer to hold the parsed edits
680    * @param fs the file system
681    * @param conf the configuration
682    * @param skipErrors indicator if CorruptedLogFileException should be thrown instead of IOException
683    * @throws IOException
684    * @throws CorruptedLogFileException if hlog is corrupted
685    */
686   private void parseHLog(final Reader in, Path path,
687 		EntryBuffers entryBuffers, final FileSystem fs,
688     final Configuration conf, boolean skipErrors)
689 	throws IOException, CorruptedLogFileException {
690     int editsCount = 0;
691     try {
692       Entry entry;
693       while ((entry = getNextLogLine(in, path, skipErrors)) != null) {
694         entryBuffers.appendEntry(entry);
695         editsCount++;
696       }
697     } catch (InterruptedException ie) {
698       IOException t = new InterruptedIOException();
699       t.initCause(ie);
700       throw t;
701     } finally {
702       LOG.debug("Pushed=" + editsCount + " entries from " + path);
703     }
704   }
705 
706   /**
707    * Create a new {@link Reader} for reading logs to split.
708    *
709    * @param fs
710    * @param file
711    * @param conf
712    * @return A new Reader instance
713    * @throws IOException
714    * @throws CorruptedLogFile
715    */
716   protected Reader getReader(FileSystem fs, FileStatus file, Configuration conf,
717       boolean skipErrors)
718       throws IOException, CorruptedLogFileException {
719     Path path = file.getPath();
720     long length = file.getLen();
721     Reader in;
722 
723 
724     // Check for possibly empty file. With appends, currently Hadoop reports a
725     // zero length even if the file has been sync'd. Revisit if HDFS-376 or
726     // HDFS-878 is committed.
727     if (length <= 0) {
728       LOG.warn("File " + path + " might be still open, length is 0");
729     }
730 
731     try {
732       FSUtils.getInstance(fs, conf).recoverFileLease(fs, path, conf);
733       try {
734         in = getReader(fs, path, conf);
735       } catch (EOFException e) {
736         if (length <= 0) {
737           // TODO should we ignore an empty, not-last log file if skip.errors
738           // is false? Either way, the caller should decide what to do. E.g.
739           // ignore if this is the last log in sequence.
740           // TODO is this scenario still possible if the log has been
741           // recovered (i.e. closed)
742           LOG.warn("Could not open " + path + " for reading. File is empty", e);
743           return null;
744         } else {
745           // EOFException being ignored
746           return null;
747         }
748       }
749     } catch (IOException e) {
750       if (!skipErrors) {
751         throw e;
752       }
753       CorruptedLogFileException t =
754         new CorruptedLogFileException("skipErrors=true Could not open hlog " +
755             path + " ignoring");
756       t.initCause(e);
757       throw t;
758     }
759     return in;
760   }
761 
762   static private Entry getNextLogLine(Reader in, Path path, boolean skipErrors)
763   throws CorruptedLogFileException, IOException {
764     try {
765       return in.next();
766     } catch (EOFException eof) {
767       // truncated files are expected if a RS crashes (see HBASE-2643)
768       LOG.info("EOF from hlog " + path + ".  continuing");
769       return null;
770     } catch (IOException e) {
771       // If the IOE resulted from bad file format,
772       // then this problem is idempotent and retrying won't help
773       if (e.getCause() != null &&
774           (e.getCause() instanceof ParseException ||
775            e.getCause() instanceof org.apache.hadoop.fs.ChecksumException)) {
776         LOG.warn("Parse exception " + e.getCause().toString() + " from hlog "
777            + path + ".  continuing");
778         return null;
779       }
780       if (!skipErrors) {
781         throw e;
782       }
783       CorruptedLogFileException t =
784         new CorruptedLogFileException("skipErrors=true Ignoring exception" +
785             " while parsing hlog " + path + ". Marking as corrupted");
786       t.initCause(e);
787       throw t;
788     }
789   }
790 
791 
792   private void writerThreadError(Throwable t) {
793     thrown.compareAndSet(null, t);
794   }
795 
796   /**
797    * Check for errors in the writer threads. If any is found, rethrow it.
798    */
799   private void checkForErrors() throws IOException {
800     Throwable thrown = this.thrown.get();
801     if (thrown == null) return;
802     if (thrown instanceof IOException) {
803       throw (IOException)thrown;
804     } else {
805       throw new RuntimeException(thrown);
806     }
807   }
808   /**
809    * Create a new {@link Writer} for writing log splits.
810    */
811   protected Writer createWriter(FileSystem fs, Path logfile, Configuration conf)
812       throws IOException {
813     return HLog.createWriter(fs, logfile, conf);
814   }
815 
816   /**
817    * Create a new {@link Reader} for reading logs to split.
818    */
819   protected Reader getReader(FileSystem fs, Path curLogFile, Configuration conf)
820       throws IOException {
821     return HLog.getReader(fs, curLogFile, conf);
822   }
823 
824   /**
825    * Class which accumulates edits and separates them into a buffer per region
826    * while simultaneously accounting RAM usage. Blocks if the RAM usage crosses
827    * a predefined threshold.
828    *
829    * Writer threads then pull region-specific buffers from this class.
830    */
831   class EntryBuffers {
832     Map<byte[], RegionEntryBuffer> buffers =
833       new TreeMap<byte[], RegionEntryBuffer>(Bytes.BYTES_COMPARATOR);
834 
835     /* Track which regions are currently in the middle of writing. We don't allow
836        an IO thread to pick up bytes from a region if we're already writing
837        data for that region in a different IO thread. */
838     Set<byte[]> currentlyWriting = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
839 
840     long totalBuffered = 0;
841     long maxHeapUsage;
842 
843     EntryBuffers(long maxHeapUsage) {
844       this.maxHeapUsage = maxHeapUsage;
845     }
846 
847     /**
848      * Append a log entry into the corresponding region buffer.
849      * Blocks if the total heap usage has crossed the specified threshold.
850      *
851      * @throws InterruptedException
852      * @throws IOException
853      */
854     void appendEntry(Entry entry) throws InterruptedException, IOException {
855       HLogKey key = entry.getKey();
856 
857       RegionEntryBuffer buffer;
858       long incrHeap;
859       synchronized (this) {
860         buffer = buffers.get(key.getEncodedRegionName());
861         if (buffer == null) {
862           buffer = new RegionEntryBuffer(key.getTablename(), key.getEncodedRegionName());
863           buffers.put(key.getEncodedRegionName(), buffer);
864         }
865         incrHeap= buffer.appendEntry(entry);        
866       }
867 
868       // If we crossed the chunk threshold, wait for more space to be available
869       synchronized (dataAvailable) {
870         totalBuffered += incrHeap;
871         while (totalBuffered > maxHeapUsage && thrown.get() == null) {
872           LOG.debug("Used " + totalBuffered + " bytes of buffered edits, waiting for IO threads...");
873           dataAvailable.wait(3000);
874         }
875         dataAvailable.notifyAll();
876       }
877       checkForErrors();
878     }
879 
880     synchronized RegionEntryBuffer getChunkToWrite() {
881       long biggestSize=0;
882       byte[] biggestBufferKey=null;
883 
884       for (Map.Entry<byte[], RegionEntryBuffer> entry : buffers.entrySet()) {
885         long size = entry.getValue().heapSize();
886         if (size > biggestSize && !currentlyWriting.contains(entry.getKey())) {
887           biggestSize = size;
888           biggestBufferKey = entry.getKey();
889         }
890       }
891       if (biggestBufferKey == null) {
892         return null;
893       }
894 
895       RegionEntryBuffer buffer = buffers.remove(biggestBufferKey);
896       currentlyWriting.add(biggestBufferKey);
897       return buffer;
898     }
899 
900     void doneWriting(RegionEntryBuffer buffer) {
901       synchronized (this) {
902         boolean removed = currentlyWriting.remove(buffer.encodedRegionName);
903         assert removed;
904       }
905       long size = buffer.heapSize();
906 
907       synchronized (dataAvailable) {
908         totalBuffered -= size;
909         // We may unblock writers
910         dataAvailable.notifyAll();
911       }
912     }
913 
914     synchronized boolean isRegionCurrentlyWriting(byte[] region) {
915       return currentlyWriting.contains(region);
916     }
917   }
918 
919   /**
920    * A buffer of some number of edits for a given region.
921    * This accumulates edits and also provides a memory optimization in order to
922    * share a single byte array instance for the table and region name.
923    * Also tracks memory usage of the accumulated edits.
924    */
925   static class RegionEntryBuffer implements HeapSize {
926     long heapInBuffer = 0;
927     List<Entry> entryBuffer;
928     byte[] tableName;
929     byte[] encodedRegionName;
930 
931     RegionEntryBuffer(byte[] table, byte[] region) {
932       this.tableName = table;
933       this.encodedRegionName = region;
934       this.entryBuffer = new LinkedList<Entry>();
935     }
936 
937     long appendEntry(Entry entry) {
938       internify(entry);
939       entryBuffer.add(entry);
940       long incrHeap = entry.getEdit().heapSize() +
941         ClassSize.align(2 * ClassSize.REFERENCE) + // HLogKey pointers
942         0; // TODO linkedlist entry
943       heapInBuffer += incrHeap;
944       return incrHeap;
945     }
946 
947     private void internify(Entry entry) {
948       HLogKey k = entry.getKey();
949       k.internTableName(this.tableName);
950       k.internEncodedRegionName(this.encodedRegionName);
951     }
952 
953     public long heapSize() {
954       return heapInBuffer;
955     }
956   }
957 
958 
959   class WriterThread extends Thread {
960     private volatile boolean shouldStop = false;
961 
962     WriterThread(int i) {
963       super("WriterThread-" + i);
964     }
965 
966     public void run()  {
967       try {
968         doRun();
969       } catch (Throwable t) {
970         LOG.error("Error in log splitting write thread", t);
971         writerThreadError(t);
972       }
973     }
974 
975     private void doRun() throws IOException {
976       LOG.debug("Writer thread " + this + ": starting");
977       while (true) {
978         RegionEntryBuffer buffer = entryBuffers.getChunkToWrite();
979         if (buffer == null) {
980           // No data currently available, wait on some more to show up
981           synchronized (dataAvailable) {
982             if (shouldStop) return;
983             try {
984               dataAvailable.wait(1000);
985             } catch (InterruptedException ie) {
986               if (!shouldStop) {
987                 throw new RuntimeException(ie);
988               }
989             }
990           }
991           continue;
992         }
993 
994         assert buffer != null;
995         try {
996           writeBuffer(buffer);
997         } finally {
998           entryBuffers.doneWriting(buffer);
999         }
1000       }
1001     }
1002 
1003 
1004     private void writeBuffer(RegionEntryBuffer buffer) throws IOException {
1005       List<Entry> entries = buffer.entryBuffer;
1006       if (entries.isEmpty()) {
1007         LOG.warn(this.getName() + " got an empty buffer, skipping");
1008         return;
1009       }
1010 
1011       WriterAndPath wap = null;
1012 
1013       long startTime = System.nanoTime();
1014       try {
1015         int editsCount = 0;
1016 
1017         for (Entry logEntry : entries) {
1018           if (wap == null) {
1019             wap = outputSink.getWriterAndPath(logEntry);
1020             if (wap == null) {
1021               // getWriterAndPath decided we don't need to write these edits
1022               // Message was already logged
1023               return;
1024             }
1025           }
1026           wap.w.append(logEntry);
1027           outputSink.updateRegionMaximumEditLogSeqNum(logEntry);
1028           editsCount++;
1029         }
1030         // Pass along summary statistics
1031         wap.incrementEdits(editsCount);
1032         wap.incrementNanoTime(System.nanoTime() - startTime);
1033       } catch (IOException e) {
1034         e = RemoteExceptionHandler.checkIOException(e);
1035         LOG.fatal(this.getName() + " Got while writing log entry to log", e);
1036         throw e;
1037       }
1038     }
1039 
1040     void finish() {
1041       synchronized (dataAvailable) {
1042         shouldStop = true;
1043         dataAvailable.notifyAll();
1044       }
1045     }
1046   }
1047 
1048   private WriterAndPath createWAP(byte[] region, Entry entry, Path rootdir,
1049       FileSystem fs, Configuration conf)
1050   throws IOException {
1051     Path regionedits = getRegionSplitEditsPath(fs, entry, rootdir, true);
1052     if (regionedits == null) {
1053       return null;
1054     }
1055     if (fs.exists(regionedits)) {
1056       LOG.warn("Found existing old edits file. It could be the "
1057           + "result of a previous failed split attempt. Deleting "
1058           + regionedits + ", length="
1059           + fs.getFileStatus(regionedits).getLen());
1060       if (!fs.delete(regionedits, false)) {
1061         LOG.warn("Failed delete of old " + regionedits);
1062       }
1063     }
1064     Writer w = createWriter(fs, regionedits, conf);
1065     LOG.debug("Creating writer path=" + regionedits + " region="
1066         + Bytes.toStringBinary(region));
1067     return (new WriterAndPath(regionedits, w));
1068   }
1069 
1070   Path convertRegionEditsToTemp(Path rootdir, Path edits, String tmpname) {
1071     List<String> components = new ArrayList<String>(10);
1072     do {
1073       components.add(edits.getName());
1074       edits = edits.getParent();
1075     } while (edits.depth() > rootdir.depth());
1076     Path ret = ZKSplitLog.getSplitLogDir(rootdir, tmpname);
1077     for (int i = components.size() - 1; i >= 0; i--) {
1078       ret = new Path(ret, components.get(i));
1079     }
1080     try {
1081       if (fs.exists(ret)) {
1082         LOG.warn("Found existing old temporary edits file. It could be the "
1083             + "result of a previous failed split attempt. Deleting "
1084             + ret + ", length="
1085             + fs.getFileStatus(ret).getLen());
1086         if (!fs.delete(ret, false)) {
1087           LOG.warn("Failed delete of old " + ret);
1088         }
1089       }
1090       Path dir = ret.getParent();
1091       if (!fs.exists(dir)) {
1092         if (!fs.mkdirs(dir)) LOG.warn("mkdir failed on " + dir);
1093       }
1094     } catch (IOException e) {
1095       LOG.warn("Could not prepare temp staging area ", e);
1096       // ignore, exceptions will be thrown elsewhere
1097     }
1098     return ret;
1099   }
1100 
1101   /**
1102    * Class that manages the output streams from the log splitting process.
1103    */
1104   class OutputSink {
1105     private final Map<byte[], WriterAndPath> logWriters = Collections.synchronizedMap(
1106           new TreeMap<byte[], WriterAndPath>(Bytes.BYTES_COMPARATOR));
1107     private final Map<byte[], Long> regionMaximumEditLogSeqNum = Collections
1108         .synchronizedMap(new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR));
1109     private final List<WriterThread> writerThreads = Lists.newArrayList();
1110 
1111     /* Set of regions which we've decided should not output edits */
1112     private final Set<byte[]> blacklistedRegions = Collections.synchronizedSet(
1113         new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR));
1114 
1115     private boolean closeAndCleanCompleted = false;
1116     
1117     private boolean logWritersClosed  = false;
1118 
1119     /**
1120      * Start the threads that will pump data from the entryBuffers
1121      * to the output files.
1122      * @return the list of started threads
1123      */
1124     synchronized void startWriterThreads(EntryBuffers entryBuffers) {
1125       // More threads could potentially write faster at the expense
1126       // of causing more disk seeks as the logs are split.
1127       // 3. After a certain setting (probably around 3) the
1128       // process will be bound on the reader in the current
1129       // implementation anyway.
1130       int numThreads = conf.getInt(
1131           "hbase.regionserver.hlog.splitlog.writer.threads", 3);
1132 
1133       for (int i = 0; i < numThreads; i++) {
1134         WriterThread t = new WriterThread(i);
1135         t.start();
1136         writerThreads.add(t);
1137       }
1138     }
1139 
1140     List<Path> finishWritingAndClose() throws IOException {
1141       LOG.info("Waiting for split writer threads to finish");
1142       try {
1143         for (WriterThread t : writerThreads) {
1144           t.finish();
1145         }
1146         for (WriterThread t : writerThreads) {
1147           try {
1148             t.join();
1149           } catch (InterruptedException ie) {
1150             throw new IOException(ie);
1151           }
1152           checkForErrors();
1153         }
1154         LOG.info("Split writers finished");
1155 
1156         return closeStreams();
1157       } finally {
1158         List<IOException> thrown = closeLogWriters(null);
1159         if (thrown != null && !thrown.isEmpty()) {
1160           throw MultipleIOException.createIOException(thrown);
1161         }
1162       }
1163     }
1164 
1165     /**
1166      * Close all of the output streams.
1167      * @return the list of paths written.
1168      */
1169     private List<Path> closeStreams() throws IOException {
1170       Preconditions.checkState(!closeAndCleanCompleted);
1171 
1172       List<Path> paths = new ArrayList<Path>();
1173       List<IOException> thrown = Lists.newArrayList();
1174       closeLogWriters(thrown);
1175       for (Map.Entry<byte[], WriterAndPath> logWritersEntry : logWriters
1176           .entrySet()) {
1177         WriterAndPath wap = logWritersEntry.getValue();
1178         Path dst = getCompletedRecoveredEditsFilePath(wap.p,
1179             regionMaximumEditLogSeqNum.get(logWritersEntry.getKey()));
1180         try {
1181           if (!dst.equals(wap.p) && fs.exists(dst)) {
1182             LOG.warn("Found existing old edits file. It could be the "
1183                 + "result of a previous failed split attempt. Deleting " + dst
1184                 + ", length=" + fs.getFileStatus(dst).getLen());
1185             if (!fs.delete(dst, false)) {
1186               LOG.warn("Failed deleting of old " + dst);
1187               throw new IOException("Failed deleting of old " + dst);
1188             }
1189           }
1190           // Skip the unit tests which create a splitter that reads and writes
1191           // the data without touching disk. TestHLogSplit#testThreading is an
1192           // example.
1193           if (fs.exists(wap.p)) {
1194             if (!fs.rename(wap.p, dst)) {
1195               throw new IOException("Failed renaming " + wap.p + " to " + dst);
1196             }
1197             LOG.debug("Rename " + wap.p + " to " + dst);
1198           }
1199         } catch (IOException ioe) {
1200           LOG.error("Couldn't rename " + wap.p + " to " + dst, ioe);
1201           thrown.add(ioe);
1202           continue;
1203         }
1204         paths.add(dst);
1205       }
1206       if (!thrown.isEmpty()) {
1207         throw MultipleIOException.createIOException(thrown);
1208       }
1209 
1210       closeAndCleanCompleted = true;
1211       return paths;
1212     }
1213     
1214     private List<IOException> closeLogWriters(List<IOException> thrown)
1215         throws IOException {
1216       if (!logWritersClosed) {
1217         if (thrown == null) {
1218           thrown = Lists.newArrayList();
1219         }
1220         for (WriterAndPath wap : logWriters.values()) {
1221           try {
1222             wap.w.close();
1223           } catch (IOException ioe) {
1224             LOG.error("Couldn't close log at " + wap.p, ioe);
1225             thrown.add(ioe);
1226             continue;
1227           }
1228           LOG.info("Closed path " + wap.p + " (wrote " + wap.editsWritten
1229               + " edits in " + (wap.nanosSpent / 1000 / 1000) + "ms)");
1230         }
1231         logWritersClosed = true;
1232       }
1233       return thrown;
1234     }
1235 
1236     /**
1237      * Get a writer and path for a log starting at the given entry.
1238      *
1239      * This function is threadsafe so long as multiple threads are always
1240      * acting on different regions.
1241      *
1242      * @return null if this region shouldn't output any logs
1243      */
1244     WriterAndPath getWriterAndPath(Entry entry) throws IOException {
1245       byte region[] = entry.getKey().getEncodedRegionName();
1246       WriterAndPath ret = logWriters.get(region);
1247       if (ret != null) {
1248         return ret;
1249       }
1250       // If we already decided that this region doesn't get any output
1251       // we don't need to check again.
1252       if (blacklistedRegions.contains(region)) {
1253         return null;
1254       }
1255       ret = createWAP(region, entry, rootDir, fs, conf);
1256       if (ret == null) {
1257         blacklistedRegions.add(region);
1258         return null;
1259       }
1260       logWriters.put(region, ret);
1261       return ret;
1262     }
1263 
1264     /**
1265      * Update region's maximum edit log SeqNum.
1266      */
1267     void updateRegionMaximumEditLogSeqNum(Entry entry) {
1268       synchronized (regionMaximumEditLogSeqNum) {
1269         Long currentMaxSeqNum=regionMaximumEditLogSeqNum.get(entry.getKey().getEncodedRegionName());
1270         if (currentMaxSeqNum == null
1271             || entry.getKey().getLogSeqNum() > currentMaxSeqNum) {
1272           regionMaximumEditLogSeqNum.put(entry.getKey().getEncodedRegionName(),
1273               entry.getKey().getLogSeqNum());
1274         }
1275       }
1276 
1277     }
1278 
1279     Long getRegionMaximumEditLogSeqNum(byte[] region) {
1280       return regionMaximumEditLogSeqNum.get(region);
1281     }
1282 
1283     /**
1284      * @return a map from encoded region ID to the number of edits written out
1285      * for that region.
1286      */
1287     private Map<byte[], Long> getOutputCounts() {
1288       TreeMap<byte[], Long> ret = new TreeMap<byte[], Long>(
1289           Bytes.BYTES_COMPARATOR);
1290       synchronized (logWriters) {
1291         for (Map.Entry<byte[], WriterAndPath> entry : logWriters.entrySet()) {
1292           ret.put(entry.getKey(), entry.getValue().editsWritten);
1293         }
1294       }
1295       return ret;
1296     }
1297   }
1298 
1299 
1300 
1301   /**
1302    *  Private data structure that wraps a Writer and its Path,
1303    *  also collecting statistics about the data written to this
1304    *  output.
1305    */
1306   private final static class WriterAndPath {
1307     final Path p;
1308     final Writer w;
1309 
1310     /* Count of edits written to this path */
1311     long editsWritten = 0;
1312     /* Number of nanos spent writing to this log */
1313     long nanosSpent = 0;
1314     
1315     /* To check whether a close has already been tried on the
1316      * writer
1317      */
1318     boolean writerClosed = false;
1319 
1320     WriterAndPath(final Path p, final Writer w) {
1321       this.p = p;
1322       this.w = w;
1323     }
1324 
1325     void incrementEdits(int edits) {
1326       editsWritten += edits;
1327     }
1328 
1329     void incrementNanoTime(long nanos) {
1330       nanosSpent += nanos;
1331     }
1332   }
1333 
1334   static class CorruptedLogFileException extends Exception {
1335     private static final long serialVersionUID = 1L;
1336     CorruptedLogFileException(String s) {
1337       super(s);
1338     }
1339   }
1340 }