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