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     long totalBytesToSplit = 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 a temporary staging area. tmpname is used to build
330    * the name of the staging area where the recovered-edits will be separated
331    * out by region and stored.
332    * <p>
333    * If the log file has N regions then N recovered.edits files will be
334    * produced. There is no buffering in this code. Instead it relies on the
335    * buffering in the SequenceFileWriter.
336    * <p>
337    * @param rootDir
338    * @param tmpname
339    * @param logfile
340    * @param fs
341    * @param conf
342    * @param reporter
343    * @return false if it is interrupted by the progress-able.
344    * @throws IOException
345    */
346   static public boolean splitLogFileToTemp(Path rootDir, String tmpname,
347       FileStatus logfile, FileSystem fs,
348       Configuration conf, CancelableProgressable reporter) throws IOException {
349     HLogSplitter s = new HLogSplitter(conf, rootDir, null, null /* oldLogDir */,
350         fs);
351     return s.splitLogFileToTemp(logfile, tmpname, reporter);
352   }
353 
354   public boolean splitLogFileToTemp(FileStatus logfile, String tmpname,
355       CancelableProgressable reporter)  throws IOException {	    
356     final Map<byte[], Object> logWriters = Collections.
357     synchronizedMap(new TreeMap<byte[], Object>(Bytes.BYTES_COMPARATOR));
358     boolean isCorrupted = false;
359     
360     Preconditions.checkState(status == null);
361     status = TaskMonitor.get().createStatus(
362         "Splitting log file " + logfile.getPath() +
363         "into a temporary staging area.");
364 
365     Object BAD_WRITER = new Object();
366 
367     boolean progress_failed = false;
368 
369     boolean skipErrors = conf.getBoolean("hbase.hlog.split.skip.errors",
370         HLog.SPLIT_SKIP_ERRORS_DEFAULT);
371     int interval = conf.getInt("hbase.splitlog.report.interval.loglines", 1024);
372     // How often to send a progress report (default 1/2 master timeout)
373     int period = conf.getInt("hbase.splitlog.report.period",
374         conf.getInt("hbase.splitlog.manager.timeout",
375             ZKSplitLog.DEFAULT_TIMEOUT) / 2);
376     Path logPath = logfile.getPath();
377     long logLength = logfile.getLen();
378     LOG.info("Splitting hlog: " + logPath + ", length=" + logLength);
379     status.setStatus("Opening log file");
380     Reader in = null;
381     try {
382       in = getReader(fs, logfile, conf, skipErrors);
383     } catch (CorruptedLogFileException e) {
384       LOG.warn("Could not get reader, corrupted log file " + logPath, e);
385       ZKSplitLog.markCorrupted(rootDir, tmpname, fs);
386       isCorrupted = true;
387     }
388     if (in == null) {
389       status.markComplete("Was nothing to split in log file");
390       LOG.warn("Nothing to split in log file " + logPath);
391       return true;
392     }
393     long t = EnvironmentEdgeManager.currentTimeMillis();
394     long last_report_at = t;
395     if (reporter != null && reporter.progress() == false) {
396       status.markComplete("Failed: reporter.progress asked us to terminate");
397       return false;
398     }
399     int editsCount = 0;
400     Entry entry;
401     try {
402       while ((entry = getNextLogLine(in,logPath, skipErrors)) != null) {
403         byte[] region = entry.getKey().getEncodedRegionName();
404         Object o = logWriters.get(region);
405         if (o == BAD_WRITER) {
406           continue;
407         }
408         WriterAndPath wap = (WriterAndPath)o;
409         if (wap == null) {
410           wap = createWAP(region, entry, rootDir, tmpname, fs, conf);
411           if (wap == null) {
412         	  // ignore edits from this region. It doesn't ezist anymore.
413         	  // It was probably already split.
414             logWriters.put(region, BAD_WRITER);
415             continue;
416           } else {
417             logWriters.put(region, wap);
418           }
419         }
420         wap.w.append(entry);
421         editsCount++;
422         if (editsCount % interval == 0) {
423           status.setStatus("Split " + editsCount + " edits");
424           long t1 = EnvironmentEdgeManager.currentTimeMillis();
425           if ((t1 - last_report_at) > period) {
426             last_report_at = t;
427             if (reporter != null && reporter.progress() == false) {
428               status.markComplete("Failed: reporter.progress asked us to terminate");
429               progress_failed = true;
430               return false;
431             }
432           }
433         }
434       }
435     } catch (CorruptedLogFileException e) {
436       LOG.warn("Could not parse, corrupted log file " + logPath, e);
437       ZKSplitLog.markCorrupted(rootDir, tmpname, fs);
438       isCorrupted = true;
439     } catch (IOException e) {
440       e = RemoteExceptionHandler.checkIOException(e);
441       throw e;
442     } finally {
443       int n = 0;
444       for (Object o : logWriters.values()) {
445         long t1 = EnvironmentEdgeManager.currentTimeMillis();
446         if ((t1 - last_report_at) > period) {
447           last_report_at = t;
448           if ((progress_failed == false) && (reporter != null) &&
449               (reporter.progress() == false)) {
450             progress_failed = true;
451           }
452         }
453         if (o == BAD_WRITER) {
454           continue;
455         }
456         n++;
457         WriterAndPath wap = (WriterAndPath)o;
458         wap.w.close();
459         LOG.debug("Closed " + wap.p);
460         Path dst = getCompletedRecoveredEditsFilePath(wap.p);
461         if (!dst.equals(wap.p) && fs.exists(dst)) {
462           LOG.warn("Found existing old edits file. It could be the "
463               + "result of a previous failed split attempt. Deleting " + dst
464               + ", length=" + fs.getFileStatus(dst).getLen());
465           if (!fs.delete(dst, false)) {
466             LOG.warn("Failed deleting of old " + dst);
467             throw new IOException("Failed deleting of old " + dst);
468           }
469         }
470         // Skip the unit tests which create a splitter that reads and writes the
471         // data without touching disk. TestHLogSplit#testThreading is an
472         // example.
473         if (fs.exists(wap.p)) {
474           if (!fs.rename(wap.p, dst)) {
475             throw new IOException("Failed renaming " + wap.p + " to " + dst);
476           }
477         }
478       }
479       String msg = ("processed " + editsCount + " edits across " + n + " regions" +
480           " threw away edits for " + (logWriters.size() - n) + " regions" +
481           " log file = " + logPath +
482           " is corrupted = " + isCorrupted);
483       LOG.info(msg);
484       status.markComplete(msg);
485     }
486     return true;
487   }
488 
489   /**
490    * Completes the work done by splitLogFileToTemp by moving the
491    * recovered.edits from the staging area to the respective region server's
492    * directories.
493    * <p>
494    * It is invoked by SplitLogManager once it knows that one of the
495    * SplitLogWorkers have completed the splitLogFileToTemp() part. If the
496    * master crashes then this function might get called multiple times.
497    * <p>
498    * @param tmpname
499    * @param conf
500    * @throws IOException
501    */
502   public static void moveRecoveredEditsFromTemp(String tmpname,
503       String logfile, Configuration conf)
504   throws IOException{
505     Path rootdir = FSUtils.getRootDir(conf);
506     Path oldLogDir = new Path(rootdir, HConstants.HREGION_OLDLOGDIR_NAME);
507     moveRecoveredEditsFromTemp(tmpname, rootdir, oldLogDir, logfile, conf);
508   }
509 
510   public static void moveRecoveredEditsFromTemp(String tmpname,
511       Path rootdir, Path oldLogDir,
512       String logfile, Configuration conf)
513   throws IOException{
514     List<Path> processedLogs = new ArrayList<Path>();
515     List<Path> corruptedLogs = new ArrayList<Path>();
516     FileSystem fs;
517     fs = rootdir.getFileSystem(conf);
518     Path logPath = new Path(logfile);
519     if (ZKSplitLog.isCorrupted(rootdir, tmpname, fs)) {
520       corruptedLogs.add(logPath);
521     } else {
522       processedLogs.add(logPath);
523     }
524     Path stagingDir = ZKSplitLog.getSplitLogDir(rootdir, tmpname);
525     List<FileStatus> files = listAll(fs, stagingDir);
526     for (FileStatus f : files) {
527       Path src = f.getPath();
528       Path dst = ZKSplitLog.stripSplitLogTempDir(rootdir, src);
529       if (ZKSplitLog.isCorruptFlagFile(dst)) {
530         continue;
531       }
532       if (fs.exists(dst)) {
533         fs.delete(dst, false);
534       } else {
535         Path dstdir = dst.getParent();
536         if (!fs.exists(dstdir)) {
537           if (!fs.mkdirs(dstdir)) LOG.warn("mkdir failed on " + dstdir);
538         }
539       }
540       fs.rename(src, dst);
541       LOG.debug(" moved " + src + " => " + dst);
542     }
543     archiveLogs(null, corruptedLogs, processedLogs,
544         oldLogDir, fs, conf);
545     fs.delete(stagingDir, true);
546     return;
547   }
548 
549   private static List<FileStatus> listAll(FileSystem fs, Path dir)
550   throws IOException {
551     List<FileStatus> fset = new ArrayList<FileStatus>(100);
552     FileStatus [] files = fs.exists(dir)? fs.listStatus(dir): null;
553     if (files != null) {
554       for (FileStatus f : files) {
555         if (f.isDir()) {
556           fset.addAll(listAll(fs, f.getPath()));
557         } else {
558           fset.add(f);
559         }
560       }
561     }
562     return fset;
563   }
564 
565 
566   /**
567    * Moves processed logs to a oldLogDir after successful processing Moves
568    * corrupted logs (any log that couldn't be successfully parsed to corruptDir
569    * (.corrupt) for later investigation
570    *
571    * @param corruptedLogs
572    * @param processedLogs
573    * @param oldLogDir
574    * @param fs
575    * @param conf
576    * @throws IOException
577    */
578   private static void archiveLogs(
579       final Path srcDir,
580       final List<Path> corruptedLogs,
581       final List<Path> processedLogs, final Path oldLogDir,
582       final FileSystem fs, final Configuration conf) throws IOException {
583     final Path corruptDir = new Path(conf.get(HConstants.HBASE_DIR), conf.get(
584         "hbase.regionserver.hlog.splitlog.corrupt.dir", ".corrupt"));
585 
586     if (!fs.mkdirs(corruptDir)) {
587       LOG.info("Unable to mkdir " + corruptDir);
588     }
589     fs.mkdirs(oldLogDir);
590 
591     for (Path corrupted : corruptedLogs) {
592       Path p = new Path(corruptDir, corrupted.getName());
593       if (!fs.rename(corrupted, p)) {
594         LOG.info("Unable to move corrupted log " + corrupted + " to " + p);
595       } else {
596         LOG.info("Moving corrupted log " + corrupted + " to " + p);
597       }
598     }
599 
600     for (Path p : processedLogs) {
601       Path newPath = HLog.getHLogArchivePath(oldLogDir, p);
602       if (!fs.rename(p, newPath)) {
603         LOG.info("Unable to move  " + p + " to " + newPath);
604       } else {
605         LOG.info("Archived processed log " + p + " to " + newPath);
606       }
607     }
608 
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    * Convert path to a file under RECOVERED_EDITS_DIR directory without
658    * RECOVERED_LOG_TMPFILE_SUFFIX
659    * @param srcPath
660    * @return dstPath without RECOVERED_LOG_TMPFILE_SUFFIX
661    */
662   static Path getCompletedRecoveredEditsFilePath(Path srcPath) {
663     String fileName = srcPath.getName();
664     if (fileName.endsWith(HLog.RECOVERED_LOG_TMPFILE_SUFFIX)) {
665       fileName = fileName.split(HLog.RECOVERED_LOG_TMPFILE_SUFFIX)[0];
666     }
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 @splitLogsMap
676    *
677    * @param logfile to split
678    * @param splitLogsMap output parameter: a map with region names as keys and a
679    * list of edits as values
680    * @param fs the filesystem
681    * @param conf the configuration
682    * @throws IOException
683    * @throws CorruptedLogFileException if hlog is corrupted
684    */
685   private void parseHLog(final Reader in, Path path,
686 		EntryBuffers entryBuffers, final FileSystem fs,
687     final Configuration conf, boolean skipErrors)
688 	throws IOException, CorruptedLogFileException {
689     int editsCount = 0;
690     try {
691       Entry entry;
692       while ((entry = getNextLogLine(in, path, skipErrors)) != null) {
693         entryBuffers.appendEntry(entry);
694         editsCount++;
695       }
696     } catch (InterruptedException ie) {
697       IOException t = new InterruptedIOException();
698       t.initCause(ie);
699       throw t;
700     } finally {
701       LOG.debug("Pushed=" + editsCount + " entries from " + path);
702     }
703   }
704 
705   /**
706    * Create a new {@link Reader} for reading logs to split.
707    *
708    * @param fs
709    * @param file
710    * @param conf
711    * @return A new Reader instance
712    * @throws IOException
713    * @throws CorruptedLogFile
714    */
715   protected Reader getReader(FileSystem fs, FileStatus file, Configuration conf,
716       boolean skipErrors)
717       throws IOException, CorruptedLogFileException {
718     Path path = file.getPath();
719     long length = file.getLen();
720     Reader in;
721 
722 
723     // Check for possibly empty file. With appends, currently Hadoop reports a
724     // zero length even if the file has been sync'd. Revisit if HDFS-376 or
725     // HDFS-878 is committed.
726     if (length <= 0) {
727       LOG.warn("File " + path + " might be still open, length is 0");
728     }
729 
730     try {
731       FSUtils.getInstance(fs, conf).recoverFileLease(fs, path, conf);
732       try {
733         in = getReader(fs, path, conf);
734       } catch (EOFException e) {
735         if (length <= 0) {
736           // TODO should we ignore an empty, not-last log file if skip.errors
737           // is false? Either way, the caller should decide what to do. E.g.
738           // ignore if this is the last log in sequence.
739           // TODO is this scenario still possible if the log has been
740           // recovered (i.e. closed)
741           LOG.warn("Could not open " + path + " for reading. File is empty", e);
742           return null;
743         } else {
744           // EOFException being ignored
745           return null;
746         }
747       }
748     } catch (IOException e) {
749       if (!skipErrors) {
750         throw e;
751       }
752       CorruptedLogFileException t =
753         new CorruptedLogFileException("skipErrors=true Could not open hlog " +
754             path + " ignoring");
755       t.initCause(e);
756       throw t;
757     }
758     return in;
759   }
760 
761   static private Entry getNextLogLine(Reader in, Path path, boolean skipErrors)
762   throws CorruptedLogFileException, IOException {
763     try {
764       return in.next();
765     } catch (EOFException eof) {
766       // truncated files are expected if a RS crashes (see HBASE-2643)
767       LOG.info("EOF from hlog " + path + ".  continuing");
768       return null;
769     } catch (IOException e) {
770       // If the IOE resulted from bad file format,
771       // then this problem is idempotent and retrying won't help
772       if (e.getCause() != null &&
773           (e.getCause() instanceof ParseException ||
774            e.getCause() instanceof org.apache.hadoop.fs.ChecksumException)) {
775         LOG.warn("Parse exception " + e.getCause().toString() + " from hlog "
776            + path + ".  continuing");
777         return null;
778       }
779       if (!skipErrors) {
780         throw e;
781       }
782       CorruptedLogFileException t =
783         new CorruptedLogFileException("skipErrors=true Ignoring exception" +
784             " while parsing hlog " + path + ". Marking as corrupted");
785       t.initCause(e);
786       throw t;
787     }
788   }
789 
790 
791   private void writerThreadError(Throwable t) {
792     thrown.compareAndSet(null, t);
793   }
794 
795   /**
796    * Check for errors in the writer threads. If any is found, rethrow it.
797    */
798   private void checkForErrors() throws IOException {
799     Throwable thrown = this.thrown.get();
800     if (thrown == null) return;
801     if (thrown instanceof IOException) {
802       throw (IOException)thrown;
803     } else {
804       throw new RuntimeException(thrown);
805     }
806   }
807   /**
808    * Create a new {@link Writer} for writing log splits.
809    */
810   protected Writer createWriter(FileSystem fs, Path logfile, Configuration conf)
811       throws IOException {
812     return HLog.createWriter(fs, logfile, conf);
813   }
814 
815   /**
816    * Create a new {@link Reader} for reading logs to split.
817    */
818   protected Reader getReader(FileSystem fs, Path curLogFile, Configuration conf)
819       throws IOException {
820     return HLog.getReader(fs, curLogFile, conf);
821   }
822 
823   /**
824    * Class which accumulates edits and separates them into a buffer per region
825    * while simultaneously accounting RAM usage. Blocks if the RAM usage crosses
826    * a predefined threshold.
827    *
828    * Writer threads then pull region-specific buffers from this class.
829    */
830   class EntryBuffers {
831     Map<byte[], RegionEntryBuffer> buffers =
832       new TreeMap<byte[], RegionEntryBuffer>(Bytes.BYTES_COMPARATOR);
833 
834     /* Track which regions are currently in the middle of writing. We don't allow
835        an IO thread to pick up bytes from a region if we're already writing
836        data for that region in a different IO thread. */
837     Set<byte[]> currentlyWriting = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
838 
839     long totalBuffered = 0;
840     long maxHeapUsage;
841 
842     EntryBuffers(long maxHeapUsage) {
843       this.maxHeapUsage = maxHeapUsage;
844     }
845 
846     /**
847      * Append a log entry into the corresponding region buffer.
848      * Blocks if the total heap usage has crossed the specified threshold.
849      *
850      * @throws InterruptedException
851      * @throws IOException
852      */
853     void appendEntry(Entry entry) throws InterruptedException, IOException {
854       HLogKey key = entry.getKey();
855 
856       RegionEntryBuffer buffer;
857       long incrHeap;
858       synchronized (this) {
859         buffer = buffers.get(key.getEncodedRegionName());
860         if (buffer == null) {
861           buffer = new RegionEntryBuffer(key.getTablename(), key.getEncodedRegionName());
862           buffers.put(key.getEncodedRegionName(), buffer);
863         }
864         incrHeap= buffer.appendEntry(entry);        
865       }
866 
867       // If we crossed the chunk threshold, wait for more space to be available
868       synchronized (dataAvailable) {
869         totalBuffered += incrHeap;
870         while (totalBuffered > maxHeapUsage && thrown.get() == null) {
871           LOG.debug("Used " + totalBuffered + " bytes of buffered edits, waiting for IO threads...");
872           dataAvailable.wait(3000);
873         }
874         dataAvailable.notifyAll();
875       }
876       checkForErrors();
877     }
878 
879     synchronized RegionEntryBuffer getChunkToWrite() {
880       long biggestSize=0;
881       byte[] biggestBufferKey=null;
882 
883       for (Map.Entry<byte[], RegionEntryBuffer> entry : buffers.entrySet()) {
884         long size = entry.getValue().heapSize();
885         if (size > biggestSize && !currentlyWriting.contains(entry.getKey())) {
886           biggestSize = size;
887           biggestBufferKey = entry.getKey();
888         }
889       }
890       if (biggestBufferKey == null) {
891         return null;
892       }
893 
894       RegionEntryBuffer buffer = buffers.remove(biggestBufferKey);
895       currentlyWriting.add(biggestBufferKey);
896       return buffer;
897     }
898 
899     void doneWriting(RegionEntryBuffer buffer) {
900       synchronized (this) {
901         boolean removed = currentlyWriting.remove(buffer.encodedRegionName);
902         assert removed;
903       }
904       long size = buffer.heapSize();
905 
906       synchronized (dataAvailable) {
907         totalBuffered -= size;
908         // We may unblock writers
909         dataAvailable.notifyAll();
910       }
911     }
912 
913     synchronized boolean isRegionCurrentlyWriting(byte[] region) {
914       return currentlyWriting.contains(region);
915     }
916   }
917 
918   /**
919    * A buffer of some number of edits for a given region.
920    * This accumulates edits and also provides a memory optimization in order to
921    * share a single byte array instance for the table and region name.
922    * Also tracks memory usage of the accumulated edits.
923    */
924   static class RegionEntryBuffer implements HeapSize {
925     long heapInBuffer = 0;
926     List<Entry> entryBuffer;
927     byte[] tableName;
928     byte[] encodedRegionName;
929 
930     RegionEntryBuffer(byte[] table, byte[] region) {
931       this.tableName = table;
932       this.encodedRegionName = region;
933       this.entryBuffer = new LinkedList<Entry>();
934     }
935 
936     long appendEntry(Entry entry) {
937       internify(entry);
938       entryBuffer.add(entry);
939       long incrHeap = entry.getEdit().heapSize() +
940         ClassSize.align(2 * ClassSize.REFERENCE) + // HLogKey pointers
941         0; // TODO linkedlist entry
942       heapInBuffer += incrHeap;
943       return incrHeap;
944     }
945 
946     private void internify(Entry entry) {
947       HLogKey k = entry.getKey();
948       k.internTableName(this.tableName);
949       k.internEncodedRegionName(this.encodedRegionName);
950     }
951 
952     public long heapSize() {
953       return heapInBuffer;
954     }
955   }
956 
957 
958   class WriterThread extends Thread {
959     private volatile boolean shouldStop = false;
960 
961     WriterThread(int i) {
962       super("WriterThread-" + i);
963     }
964 
965     public void run()  {
966       try {
967         doRun();
968       } catch (Throwable t) {
969         LOG.error("Error in log splitting write thread", t);
970         writerThreadError(t);
971       }
972     }
973 
974     private void doRun() throws IOException {
975       LOG.debug("Writer thread " + this + ": starting");
976       while (true) {
977         RegionEntryBuffer buffer = entryBuffers.getChunkToWrite();
978         if (buffer == null) {
979           // No data currently available, wait on some more to show up
980           synchronized (dataAvailable) {
981             if (shouldStop) return;
982             try {
983               dataAvailable.wait(1000);
984             } catch (InterruptedException ie) {
985               if (!shouldStop) {
986                 throw new RuntimeException(ie);
987               }
988             }
989           }
990           continue;
991         }
992 
993         assert buffer != null;
994         try {
995           writeBuffer(buffer);
996         } finally {
997           entryBuffers.doneWriting(buffer);
998         }
999       }
1000     }
1001 
1002     private void writeBuffer(RegionEntryBuffer buffer) throws IOException {
1003       List<Entry> entries = buffer.entryBuffer;
1004       if (entries.isEmpty()) {
1005         LOG.warn(this.getName() + " got an empty buffer, skipping");
1006         return;
1007       }
1008 
1009       WriterAndPath wap = null;
1010 
1011       long startTime = System.nanoTime();
1012       try {
1013         int editsCount = 0;
1014 
1015         for (Entry logEntry : entries) {
1016           if (wap == null) {
1017             wap = outputSink.getWriterAndPath(logEntry);
1018             if (wap == null) {
1019               // getWriterAndPath decided we don't need to write these edits
1020               // Message was already logged
1021               return;
1022             }
1023           }
1024           wap.w.append(logEntry);
1025           editsCount++;
1026         }
1027         // Pass along summary statistics
1028         wap.incrementEdits(editsCount);
1029         wap.incrementNanoTime(System.nanoTime() - startTime);
1030       } catch (IOException e) {
1031         e = RemoteExceptionHandler.checkIOException(e);
1032         LOG.fatal(this.getName() + " Got while writing log entry to log", e);
1033         throw e;
1034       }
1035     }
1036 
1037     void finish() {
1038       synchronized (dataAvailable) {
1039         shouldStop = true;
1040         dataAvailable.notifyAll();
1041       }
1042     }
1043   }
1044 
1045   private WriterAndPath createWAP(byte[] region, Entry entry,
1046       Path rootdir, String tmpname, FileSystem fs, Configuration conf)
1047   throws IOException {
1048     Path regionedits = getRegionSplitEditsPath(fs, entry, rootdir,
1049         tmpname==null);
1050     if (regionedits == null) {
1051       return null;
1052     }
1053     if ((tmpname == null) && fs.exists(regionedits)) {
1054       LOG.warn("Found existing old edits file. It could be the "
1055           + "result of a previous failed split attempt. Deleting "
1056           + regionedits + ", length="
1057           + fs.getFileStatus(regionedits).getLen());
1058       if (!fs.delete(regionedits, false)) {
1059         LOG.warn("Failed delete of old " + regionedits);
1060       }
1061     }
1062     Path editsfile;
1063     if (tmpname != null) {
1064       // During distributed log splitting the output by each
1065       // SplitLogWorker is written to a temporary area.
1066       editsfile = convertRegionEditsToTemp(rootdir, regionedits, tmpname);
1067     } else {
1068       editsfile = regionedits;
1069     }
1070     Writer w = createWriter(fs, editsfile, conf);
1071     LOG.debug("Creating writer path=" + editsfile + " region="
1072         + Bytes.toStringBinary(region));
1073     return (new WriterAndPath(editsfile, w));
1074   }
1075 
1076   Path convertRegionEditsToTemp(Path rootdir, Path edits, String tmpname) {
1077     List<String> components = new ArrayList<String>(10);
1078     do {
1079       components.add(edits.getName());
1080       edits = edits.getParent();
1081     } while (edits.depth() > rootdir.depth());
1082     Path ret = ZKSplitLog.getSplitLogDir(rootdir, tmpname);
1083     for (int i = components.size() - 1; i >= 0; i--) {
1084       ret = new Path(ret, components.get(i));
1085     }
1086     try {
1087       if (fs.exists(ret)) {
1088         LOG.warn("Found existing old temporary edits file. It could be the "
1089             + "result of a previous failed split attempt. Deleting "
1090             + ret + ", length="
1091             + fs.getFileStatus(ret).getLen());
1092         if (!fs.delete(ret, false)) {
1093           LOG.warn("Failed delete of old " + ret);
1094         }
1095       }
1096       Path dir = ret.getParent();
1097       if (!fs.exists(dir)) {
1098         if (!fs.mkdirs(dir)) LOG.warn("mkdir failed on " + dir);
1099       }
1100     } catch (IOException e) {
1101       LOG.warn("Could not prepare temp staging area ", e);
1102       // ignore, exceptions will be thrown elsewhere
1103     }
1104     return ret;
1105   }
1106 
1107   /**
1108    * Class that manages the output streams from the log splitting process.
1109    */
1110   class OutputSink {
1111     private final Map<byte[], WriterAndPath> logWriters = Collections.synchronizedMap(
1112           new TreeMap<byte[], WriterAndPath>(Bytes.BYTES_COMPARATOR));
1113     private final List<WriterThread> writerThreads = Lists.newArrayList();
1114 
1115     /* Set of regions which we've decided should not output edits */
1116     private final Set<byte[]> blacklistedRegions = Collections.synchronizedSet(
1117         new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR));
1118 
1119     private boolean hasClosed = false;
1120 
1121     /**
1122      * Start the threads that will pump data from the entryBuffers
1123      * to the output files.
1124      * @return the list of started threads
1125      */
1126     synchronized void startWriterThreads(EntryBuffers entryBuffers) {
1127       // More threads could potentially write faster at the expense
1128       // of causing more disk seeks as the logs are split.
1129       // 3. After a certain setting (probably around 3) the
1130       // process will be bound on the reader in the current
1131       // implementation anyway.
1132       int numThreads = conf.getInt(
1133           "hbase.regionserver.hlog.splitlog.writer.threads", 3);
1134 
1135       for (int i = 0; i < numThreads; i++) {
1136         WriterThread t = new WriterThread(i);
1137         t.start();
1138         writerThreads.add(t);
1139       }
1140     }
1141 
1142     List<Path> finishWritingAndClose() throws IOException {
1143       LOG.info("Waiting for split writer threads to finish");
1144       for (WriterThread t : writerThreads) {
1145         t.finish();
1146       }
1147       for (WriterThread t: writerThreads) {
1148         try {
1149           t.join();
1150         } catch (InterruptedException ie) {
1151           throw new IOException(ie);
1152         }
1153         checkForErrors();
1154       }
1155       LOG.info("Split writers finished");
1156 
1157       return closeStreams();
1158     }
1159 
1160     /**
1161      * Close all of the output streams.
1162      * @return the list of paths written.
1163      */
1164     private List<Path> closeStreams() throws IOException {
1165       Preconditions.checkState(!hasClosed);
1166 
1167       List<Path> paths = new ArrayList<Path>();
1168       List<IOException> thrown = Lists.newArrayList();
1169 
1170       for (WriterAndPath wap : logWriters.values()) {
1171         try {
1172           wap.w.close();
1173         } catch (IOException ioe) {
1174           LOG.error("Couldn't close log at " + wap.p, ioe);
1175           thrown.add(ioe);
1176           continue;
1177         }
1178         LOG.info("Closed path " + wap.p +" (wrote " + wap.editsWritten + " edits in "
1179             + (wap.nanosSpent / 1000/ 1000) + "ms)");
1180         Path dst = getCompletedRecoveredEditsFilePath(wap.p);
1181         try {
1182           if (!dst.equals(wap.p) && fs.exists(dst)) {
1183             LOG.warn("Found existing old edits file. It could be the "
1184                 + "result of a previous failed split attempt. Deleting " + dst
1185                 + ", length=" + fs.getFileStatus(dst).getLen());
1186             if (!fs.delete(dst, false)) {
1187               LOG.warn("Failed deleting of old " + dst);
1188               throw new IOException("Failed deleting of old " + dst);
1189             }
1190           }
1191           // Skip the unit tests which create a splitter that reads and writes
1192           // the data without touching disk. TestHLogSplit#testThreading is an
1193           // example.
1194           if (fs.exists(wap.p)) {
1195             if (!fs.rename(wap.p, dst)) {
1196               throw new IOException("Failed renaming " + wap.p + " to " + dst);
1197             }
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       hasClosed = true;
1211       return paths;
1212     }
1213 
1214     /**
1215      * Get a writer and path for a log starting at the given entry.
1216      *
1217      * This function is threadsafe so long as multiple threads are always
1218      * acting on different regions.
1219      *
1220      * @return null if this region shouldn't output any logs
1221      */
1222     WriterAndPath getWriterAndPath(Entry entry) throws IOException {
1223       byte region[] = entry.getKey().getEncodedRegionName();
1224       WriterAndPath ret = logWriters.get(region);
1225       if (ret != null) {
1226         return ret;
1227       }
1228       // If we already decided that this region doesn't get any output
1229       // we don't need to check again.
1230       if (blacklistedRegions.contains(region)) {
1231         return null;
1232       }
1233       ret = createWAP(region, entry, rootDir, null, fs, conf);
1234       if (ret == null) {
1235         blacklistedRegions.add(region);
1236         return null;
1237       }
1238       logWriters.put(region, ret);
1239       return ret;
1240     }
1241 
1242     /**
1243      * @return a map from encoded region ID to the number of edits written out
1244      * for that region.
1245      */
1246     private Map<byte[], Long> getOutputCounts() {
1247       TreeMap<byte[], Long> ret = new TreeMap<byte[], Long>(
1248           Bytes.BYTES_COMPARATOR);
1249       synchronized (logWriters) {
1250         for (Map.Entry<byte[], WriterAndPath> entry : logWriters.entrySet()) {
1251           ret.put(entry.getKey(), entry.getValue().editsWritten);
1252         }
1253       }
1254       return ret;
1255     }
1256   }
1257 
1258 
1259 
1260   /**
1261    *  Private data structure that wraps a Writer and its Path,
1262    *  also collecting statistics about the data written to this
1263    *  output.
1264    */
1265   private final static class WriterAndPath {
1266     final Path p;
1267     final Writer w;
1268 
1269     /* Count of edits written to this path */
1270     long editsWritten = 0;
1271     /* Number of nanos spent writing to this log */
1272     long nanosSpent = 0;
1273 
1274     WriterAndPath(final Path p, final Writer w) {
1275       this.p = p;
1276       this.w = w;
1277     }
1278 
1279     void incrementEdits(int edits) {
1280       editsWritten += edits;
1281     }
1282 
1283     void incrementNanoTime(long nanos) {
1284       nanosSpent += nanos;
1285     }
1286   }
1287 
1288   static class CorruptedLogFileException extends Exception {
1289     private static final long serialVersionUID = 1L;
1290     CorruptedLogFileException(String s) {
1291       super(s);
1292     }
1293   }
1294 }