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