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