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