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.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
78
79
80
81 public class HLogSplitter {
82 private static final String LOG_SPLITTER_IMPL = "hbase.hlog.splitter.impl";
83
84
85
86
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
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
106
107 OutputSink outputSink;
108 EntryBuffers entryBuffers;
109
110
111
112 protected AtomicReference<Throwable> thrown = new AtomicReference<Throwable>();
113
114
115
116 Object dataAvailable = new Object();
117
118 private MonitoredTask status;
119
120
121 private DistributedLogSplittingHelper distributedLogSplittingHelper = null;
122
123
124 protected final LastSequenceId sequenceIdChecker;
125
126
127
128
129
130
131
132
133
134
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
144
145
146
147
148
149
150
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);
168 ctorArgs.add(Path.class);
169 ctorArgs.add(Path.class);
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
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
221
222
223
224
225
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
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
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
269
270 public long getTime() {
271 return this.splitTime;
272 }
273
274
275
276
277 public long getSize() {
278 return this.splitSize;
279 }
280
281
282
283
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
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
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
339
340
341
342
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
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
391
392
393
394
395
396
397
398
399
400
401
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
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
450
451 int editsCount = 0;
452 Entry entry;
453 try {
454 while ((entry = getNextLogLine(in, logPath, skipErrors)) != null) {
455 entryBuffers.appendEntry(entry);
456 editsCount++;
457
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
498
499
500
501
502
503
504
505
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
533
534
535
536
537
538
539
540
541
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
557
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
581
582 if (srcDir != null && !fs.delete(srcDir, true)) {
583 throw new IOException("Unable to delete src dir: " + srcDir);
584 }
585 }
586
587
588
589
590
591
592
593
594
595
596
597
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
618
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
631
632
633
634
635
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
649
650
651
652
653
654
655
656
657
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
681
682
683
684
685
686
687
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
698
699
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
711
712
713
714
715 LOG.warn("Could not open " + path + " for reading. File is empty", e);
716 return null;
717 } else {
718
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
741 LOG.info("EOF from hlog " + path + ". continuing");
742 return null;
743 } catch (IOException e) {
744
745
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
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
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
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
799
800
801
802
803
804 class EntryBuffers {
805 Map<byte[], RegionEntryBuffer> buffers =
806 new TreeMap<byte[], RegionEntryBuffer>(Bytes.BYTES_COMPARATOR);
807
808
809
810
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
822
823
824
825
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
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
883 dataAvailable.notifyAll();
884 }
885 }
886
887 synchronized boolean isRegionCurrentlyWriting(byte[] region) {
888 return currentlyWriting.contains(region);
889 }
890 }
891
892
893
894
895
896
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) +
915 0;
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
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
996
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
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
1053
1054
1055 private boolean reportProgressIfIsDistributedLogSplitting() {
1056 if (this.distributedLogSplittingHelper != null) {
1057 return distributedLogSplittingHelper.reportProgress();
1058 } else {
1059 return true;
1060 }
1061 }
1062
1063
1064
1065
1066
1067 class DistributedLogSplittingHelper {
1068
1069 private final CancelableProgressable splitReporter;
1070
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
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
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
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
1125
1126
1127
1128
1129 numThreads = conf.getInt(
1130 "hbase.regionserver.hlog.splitlog.writer.threads", 3);
1131 }
1132
1133
1134
1135
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
1148
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
1185
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
1229
1230
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
1321
1322
1323
1324
1325
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
1334
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
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
1367
1368
1369 synchronized (regionLastFlushedSequenceIds) {
1370 lastFlushedSequenceId = regionLastFlushedSequenceIds.get(region);
1371 }
1372 if (lastFlushedSequenceId == null) {
1373 lastFlushedSequenceId = sequenceIdChecker.getLastSequenceId(region);
1374 synchronized (regionLastFlushedSequenceIds) {
1375
1376 regionLastFlushedSequenceIds.put(region, lastFlushedSequenceId);
1377 }
1378 }
1379 }
1380 return lastFlushedSequenceId >= logSeqNum;
1381 }
1382
1383
1384
1385
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
1403
1404
1405
1406 private final static class WriterAndPath {
1407 final Path p;
1408 final Writer w;
1409
1410
1411 long editsWritten = 0;
1412
1413 long editsSkipped = 0;
1414
1415 long nanosSpent = 0;
1416
1417
1418
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 }