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.FileNotFoundException;
23 import java.io.IOException;
24 import java.io.InterruptedIOException;
25 import java.text.ParseException;
26 import java.util.ArrayList;
27 import java.util.Collections;
28 import java.util.HashSet;
29 import java.util.LinkedList;
30 import java.util.List;
31 import java.util.Map;
32 import java.util.Set;
33 import java.util.TreeMap;
34 import java.util.TreeSet;
35 import java.util.UUID;
36 import java.util.concurrent.Callable;
37 import java.util.concurrent.CompletionService;
38 import java.util.concurrent.ConcurrentHashMap;
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.AtomicBoolean;
46 import java.util.concurrent.atomic.AtomicLong;
47 import java.util.concurrent.atomic.AtomicReference;
48
49 import org.apache.commons.logging.Log;
50 import org.apache.commons.logging.LogFactory;
51 import org.apache.hadoop.classification.InterfaceAudience;
52 import org.apache.hadoop.conf.Configuration;
53 import org.apache.hadoop.fs.FileStatus;
54 import org.apache.hadoop.fs.FileSystem;
55 import org.apache.hadoop.fs.Path;
56 import org.apache.hadoop.hbase.Cell;
57 import org.apache.hadoop.hbase.CellScanner;
58 import org.apache.hadoop.hbase.CellUtil;
59 import org.apache.hadoop.hbase.HConstants;
60 import org.apache.hadoop.hbase.HRegionInfo;
61 import org.apache.hadoop.hbase.HRegionLocation;
62 import org.apache.hadoop.hbase.KeyValue;
63 import org.apache.hadoop.hbase.KeyValueUtil;
64 import org.apache.hadoop.hbase.RemoteExceptionHandler;
65 import org.apache.hadoop.hbase.ServerName;
66 import org.apache.hadoop.hbase.TableName;
67 import org.apache.hadoop.hbase.TableNotFoundException;
68 import org.apache.hadoop.hbase.client.ConnectionUtils;
69 import org.apache.hadoop.hbase.client.Delete;
70 import org.apache.hadoop.hbase.client.HConnection;
71 import org.apache.hadoop.hbase.client.HConnectionManager;
72 import org.apache.hadoop.hbase.client.Mutation;
73 import org.apache.hadoop.hbase.client.Put;
74 import org.apache.hadoop.hbase.exceptions.RegionOpeningException;
75 import org.apache.hadoop.hbase.io.HeapSize;
76 import org.apache.hadoop.hbase.master.SplitLogManager;
77 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
78 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
79 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
80 import org.apache.hadoop.hbase.protobuf.RequestConverter;
81 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService.BlockingInterface;
82 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
83 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
84 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.WALEntry;
85 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
86 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
87 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey;
88 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionStoreSequenceIds;
89 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.StoreSequenceId;
90 import org.apache.hadoop.hbase.regionserver.HRegion;
91 import org.apache.hadoop.hbase.regionserver.LastSequenceId;
92 import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry;
93 import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
94 import org.apache.hadoop.hbase.regionserver.wal.HLog.Writer;
95 import org.apache.hadoop.hbase.util.Bytes;
96 import org.apache.hadoop.hbase.util.CancelableProgressable;
97 import org.apache.hadoop.hbase.util.ClassSize;
98 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
99 import org.apache.hadoop.hbase.util.FSUtils;
100 import org.apache.hadoop.hbase.util.Pair;
101 import org.apache.hadoop.hbase.util.Threads;
102 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
103 import org.apache.hadoop.hbase.zookeeper.ZKTable;
104 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
105 import org.apache.hadoop.io.MultipleIOException;
106 import org.apache.zookeeper.KeeperException;
107
108 import com.google.common.base.Preconditions;
109 import com.google.common.collect.Lists;
110 import com.google.protobuf.ServiceException;
111
112
113
114
115
116
117 @InterfaceAudience.Private
118 public class HLogSplitter {
119 static final Log LOG = LogFactory.getLog(HLogSplitter.class);
120
121
122 protected final Path rootDir;
123 protected final FileSystem fs;
124 protected final Configuration conf;
125
126
127
128 OutputSink outputSink;
129 EntryBuffers entryBuffers;
130
131 private Set<TableName> disablingOrDisabledTables =
132 new HashSet<TableName>();
133 private ZooKeeperWatcher watcher;
134
135
136
137 protected AtomicReference<Throwable> thrown = new AtomicReference<Throwable>();
138
139
140
141 final Object dataAvailable = new Object();
142
143 private MonitoredTask status;
144
145
146 protected final LastSequenceId sequenceIdChecker;
147
148 protected boolean distributedLogReplay;
149
150
151 protected Map<String, Long> lastFlushedSequenceIds = new ConcurrentHashMap<String, Long>();
152
153
154 protected Map<String, Map<byte[], Long>> regionMaxSeqIdInStores =
155 new ConcurrentHashMap<String, Map<byte[], Long>>();
156
157
158 protected String failedServerName = "";
159
160
161 private final int numWriterThreads;
162
163
164 private final int minBatchSize;
165
166 HLogSplitter(Configuration conf, Path rootDir,
167 FileSystem fs, LastSequenceId idChecker, ZooKeeperWatcher zkw) {
168 this.conf = conf;
169 this.rootDir = rootDir;
170 this.fs = fs;
171 this.sequenceIdChecker = idChecker;
172 this.watcher = zkw;
173
174 entryBuffers = new EntryBuffers(
175 conf.getInt("hbase.regionserver.hlog.splitlog.buffersize",
176 128*1024*1024));
177
178
179
180 this.minBatchSize = conf.getInt("hbase.regionserver.wal.logreplay.batch.size", 64);
181 this.distributedLogReplay = this.conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY,
182 HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG);
183
184 this.numWriterThreads = conf.getInt("hbase.regionserver.hlog.splitlog.writer.threads", 3);
185 if (zkw != null && this.distributedLogReplay) {
186 outputSink = new LogReplayOutputSink(numWriterThreads);
187 } else {
188 if (this.distributedLogReplay) {
189 LOG.info("ZooKeeperWatcher is passed in as NULL so disable distrubitedLogRepaly.");
190 }
191 this.distributedLogReplay = false;
192 outputSink = new LogRecoveredEditsOutputSink(numWriterThreads);
193 }
194 }
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213 public static boolean splitLogFile(Path rootDir, FileStatus logfile, FileSystem fs,
214 Configuration conf, CancelableProgressable reporter, LastSequenceId idChecker,
215 ZooKeeperWatcher zkw) throws IOException {
216 HLogSplitter s = new HLogSplitter(conf, rootDir, fs, idChecker, zkw);
217 return s.splitLogFile(logfile, reporter);
218 }
219
220
221
222
223
224 public static List<Path> split(Path rootDir, Path logDir, Path oldLogDir,
225 FileSystem fs, Configuration conf) throws IOException {
226 FileStatus[] logfiles = fs.listStatus(logDir);
227 List<Path> splits = new ArrayList<Path>();
228 if (logfiles != null && logfiles.length > 0) {
229 for (FileStatus logfile: logfiles) {
230 HLogSplitter s = new HLogSplitter(conf, rootDir, fs, null, null);
231 if (s.splitLogFile(logfile, null)) {
232 finishSplitLogFile(rootDir, oldLogDir, logfile.getPath(), conf);
233 if (s.outputSink.splits != null) {
234 splits.addAll(s.outputSink.splits);
235 }
236 }
237 }
238 }
239 if (!fs.delete(logDir, true)) {
240 throw new IOException("Unable to delete src dir: " + logDir);
241 }
242 return splits;
243 }
244
245
246 boolean splitLogFile(FileStatus logfile,
247 CancelableProgressable reporter) throws IOException {
248 boolean isCorrupted = false;
249 Preconditions.checkState(status == null);
250 boolean skipErrors = conf.getBoolean("hbase.hlog.split.skip.errors",
251 HLog.SPLIT_SKIP_ERRORS_DEFAULT);
252 int interval = conf.getInt("hbase.splitlog.report.interval.loglines", 1024);
253 Path logPath = logfile.getPath();
254 boolean outputSinkStarted = false;
255 boolean progress_failed = false;
256 int editsCount = 0;
257 int editsSkipped = 0;
258
259 try {
260 status = TaskMonitor.get().createStatus(
261 "Splitting log file " + logfile.getPath() +
262 "into a temporary staging area.");
263 long logLength = logfile.getLen();
264 LOG.info("Splitting hlog: " + logPath + ", length=" + logLength);
265 LOG.info("DistributedLogReplay = " + this.distributedLogReplay);
266 status.setStatus("Opening log file");
267 if (reporter != null && !reporter.progress()) {
268 progress_failed = true;
269 return false;
270 }
271 Reader in = null;
272 try {
273 in = getReader(fs, logfile, conf, skipErrors, reporter);
274 } catch (CorruptedLogFileException e) {
275 LOG.warn("Could not get reader, corrupted log file " + logPath, e);
276 ZKSplitLog.markCorrupted(rootDir, logfile.getPath().getName(), fs);
277 isCorrupted = true;
278 }
279 if (in == null) {
280 status.markComplete("Was nothing to split in log file");
281 LOG.warn("Nothing to split in log file " + logPath);
282 return true;
283 }
284 if(watcher != null) {
285 try {
286 disablingOrDisabledTables = ZKTable.getDisabledOrDisablingTables(watcher);
287 } catch (KeeperException e) {
288 throw new IOException("Can't get disabling/disabled tables", e);
289 }
290 }
291 int numOpenedFilesBeforeReporting = conf.getInt("hbase.splitlog.report.openedfiles", 3);
292 int numOpenedFilesLastCheck = 0;
293 outputSink.setReporter(reporter);
294 outputSink.startWriterThreads();
295 outputSinkStarted = true;
296 Entry entry;
297 Long lastFlushedSequenceId = -1L;
298 ServerName serverName = HLogUtil.getServerNameFromHLogDirectoryName(logPath);
299 failedServerName = (serverName == null) ? "" : serverName.getServerName();
300 while ((entry = getNextLogLine(in, logPath, skipErrors)) != null) {
301 byte[] region = entry.getKey().getEncodedRegionName();
302 String key = Bytes.toString(region);
303 lastFlushedSequenceId = lastFlushedSequenceIds.get(key);
304 if (lastFlushedSequenceId == null) {
305 if (this.distributedLogReplay) {
306 RegionStoreSequenceIds ids =
307 SplitLogManager.getRegionFlushedSequenceId(this.watcher, failedServerName, key);
308 if (ids != null) {
309 lastFlushedSequenceId = ids.getLastFlushedSequenceId();
310 }
311 } else if (sequenceIdChecker != null) {
312 lastFlushedSequenceId = sequenceIdChecker.getLastSequenceId(region);
313 }
314 if (lastFlushedSequenceId == null) {
315 lastFlushedSequenceId = -1L;
316 }
317 lastFlushedSequenceIds.put(key, lastFlushedSequenceId);
318 }
319 if (lastFlushedSequenceId >= entry.getKey().getLogSeqNum()) {
320 editsSkipped++;
321 continue;
322 }
323 entryBuffers.appendEntry(entry);
324 editsCount++;
325 int moreWritersFromLastCheck = this.getNumOpenWriters() - numOpenedFilesLastCheck;
326
327 if (editsCount % interval == 0
328 || moreWritersFromLastCheck > numOpenedFilesBeforeReporting) {
329 numOpenedFilesLastCheck = this.getNumOpenWriters();
330 String countsStr = (editsCount - (editsSkipped + outputSink.getSkippedEdits()))
331 + " edits, skipped " + editsSkipped + " edits.";
332 status.setStatus("Split " + countsStr);
333 if (reporter != null && !reporter.progress()) {
334 progress_failed = true;
335 return false;
336 }
337 }
338 }
339 } catch (InterruptedException ie) {
340 IOException iie = new InterruptedIOException();
341 iie.initCause(ie);
342 throw iie;
343 } catch (CorruptedLogFileException e) {
344 LOG.warn("Could not parse, corrupted log file " + logPath, e);
345 ZKSplitLog.markCorrupted(rootDir, logfile.getPath().getName(), fs);
346 isCorrupted = true;
347 } catch (IOException e) {
348 e = RemoteExceptionHandler.checkIOException(e);
349 throw e;
350 } finally {
351 LOG.info("Finishing writing output logs and closing down.");
352 if (outputSinkStarted) {
353 progress_failed = outputSink.finishWritingAndClose() == null;
354 }
355 String msg = "Processed " + editsCount + " edits across "
356 + outputSink.getNumberOfRecoveredRegions() + " regions; log file=" + logPath
357 + " is corrupted = " + isCorrupted + " progress failed = " + progress_failed;
358 LOG.info(msg);
359 status.markComplete(msg);
360 }
361 return !progress_failed;
362 }
363
364
365
366
367
368
369
370
371
372
373
374
375 public static void finishSplitLogFile(String logfile,
376 Configuration conf) throws IOException {
377 Path rootdir = FSUtils.getRootDir(conf);
378 Path oldLogDir = new Path(rootdir, HConstants.HREGION_OLDLOGDIR_NAME);
379 Path logPath;
380 if (FSUtils.isStartingWithPath(rootdir, logfile)) {
381 logPath = new Path(logfile);
382 } else {
383 logPath = new Path(rootdir, logfile);
384 }
385 finishSplitLogFile(rootdir, oldLogDir, logPath, conf);
386 }
387
388 static void finishSplitLogFile(Path rootdir, Path oldLogDir,
389 Path logPath, Configuration conf) throws IOException {
390 List<Path> processedLogs = new ArrayList<Path>();
391 List<Path> corruptedLogs = new ArrayList<Path>();
392 FileSystem fs;
393 fs = rootdir.getFileSystem(conf);
394 if (ZKSplitLog.isCorrupted(rootdir, logPath.getName(), fs)) {
395 corruptedLogs.add(logPath);
396 } else {
397 processedLogs.add(logPath);
398 }
399 archiveLogs(corruptedLogs, processedLogs, oldLogDir, fs, conf);
400 Path stagingDir = ZKSplitLog.getSplitLogDir(rootdir, logPath.getName());
401 fs.delete(stagingDir, true);
402 }
403
404
405
406
407
408
409
410
411
412
413
414
415
416 private static void archiveLogs(
417 final List<Path> corruptedLogs,
418 final List<Path> processedLogs, final Path oldLogDir,
419 final FileSystem fs, final Configuration conf) throws IOException {
420 final Path corruptDir = new Path(FSUtils.getRootDir(conf), conf.get(
421 "hbase.regionserver.hlog.splitlog.corrupt.dir", HConstants.CORRUPT_DIR_NAME));
422
423 if (!fs.mkdirs(corruptDir)) {
424 LOG.info("Unable to mkdir " + corruptDir);
425 }
426 fs.mkdirs(oldLogDir);
427
428
429
430 for (Path corrupted : corruptedLogs) {
431 Path p = new Path(corruptDir, corrupted.getName());
432 if (fs.exists(corrupted)) {
433 if (!fs.rename(corrupted, p)) {
434 LOG.warn("Unable to move corrupted log " + corrupted + " to " + p);
435 } else {
436 LOG.warn("Moved corrupted log " + corrupted + " to " + p);
437 }
438 }
439 }
440
441 for (Path p : processedLogs) {
442 Path newPath = FSHLog.getHLogArchivePath(oldLogDir, p);
443 if (fs.exists(p)) {
444 if (!FSUtils.renameAndSetModifyTime(fs, p, newPath)) {
445 LOG.warn("Unable to move " + p + " to " + newPath);
446 } else {
447 LOG.debug("Archived processed log " + p + " to " + newPath);
448 }
449 }
450 }
451 }
452
453
454
455
456
457
458
459
460
461
462
463
464
465 @SuppressWarnings("deprecation")
466 static Path getRegionSplitEditsPath(final FileSystem fs,
467 final Entry logEntry, final Path rootDir, boolean isCreate)
468 throws IOException {
469 Path tableDir = FSUtils.getTableDir(rootDir, logEntry.getKey().getTablename());
470 String encodedRegionName = Bytes.toString(logEntry.getKey().getEncodedRegionName());
471 Path regiondir = HRegion.getRegionDir(tableDir, encodedRegionName);
472 Path dir = HLogUtil.getRegionDirRecoveredEditsDir(regiondir);
473
474 if (!fs.exists(regiondir)) {
475 LOG.info("This region's directory doesn't exist: "
476 + regiondir.toString() + ". It is very likely that it was" +
477 " already split so it's safe to discard those edits.");
478 return null;
479 }
480 if (fs.exists(dir) && fs.isFile(dir)) {
481 Path tmp = new Path("/tmp");
482 if (!fs.exists(tmp)) {
483 fs.mkdirs(tmp);
484 }
485 tmp = new Path(tmp,
486 HConstants.RECOVERED_EDITS_DIR + "_" + encodedRegionName);
487 LOG.warn("Found existing old file: " + dir + ". It could be some "
488 + "leftover of an old installation. It should be a folder instead. "
489 + "So moving it to " + tmp);
490 if (!fs.rename(dir, tmp)) {
491 LOG.warn("Failed to sideline old file " + dir);
492 }
493 }
494
495 if (isCreate && !fs.exists(dir)) {
496 if (!fs.mkdirs(dir)) LOG.warn("mkdir failed on " + dir);
497 }
498
499
500 String fileName = formatRecoveredEditsFileName(logEntry.getKey().getLogSeqNum());
501 fileName = getTmpRecoveredEditsFileName(fileName);
502 return new Path(dir, fileName);
503 }
504
505 static String getTmpRecoveredEditsFileName(String fileName) {
506 return fileName + HLog.RECOVERED_LOG_TMPFILE_SUFFIX;
507 }
508
509
510
511
512
513
514
515
516
517 static Path getCompletedRecoveredEditsFilePath(Path srcPath,
518 Long maximumEditLogSeqNum) {
519 String fileName = formatRecoveredEditsFileName(maximumEditLogSeqNum);
520 return new Path(srcPath.getParent(), fileName);
521 }
522
523 static String formatRecoveredEditsFileName(final long seqid) {
524 return String.format("%019d", seqid);
525 }
526
527
528
529
530
531
532
533
534
535
536
537 protected Reader getReader(FileSystem fs, FileStatus file, Configuration conf,
538 boolean skipErrors, CancelableProgressable reporter)
539 throws IOException, CorruptedLogFileException {
540 Path path = file.getPath();
541 long length = file.getLen();
542 Reader in;
543
544
545
546
547 if (length <= 0) {
548 LOG.warn("File " + path + " might be still open, length is 0");
549 }
550
551 try {
552 FSUtils.getInstance(fs, conf).recoverFileLease(fs, path, conf, reporter);
553 try {
554 in = getReader(fs, path, conf, reporter);
555 } catch (EOFException e) {
556 if (length <= 0) {
557
558
559
560
561
562 LOG.warn("Could not open " + path + " for reading. File is empty", e);
563 return null;
564 } else {
565
566 return null;
567 }
568 }
569 } catch (IOException e) {
570 if (e instanceof FileNotFoundException) {
571
572 LOG.warn("File " + path + " doesn't exist anymore.", e);
573 return null;
574 }
575 if (!skipErrors || e instanceof InterruptedIOException) {
576 throw e;
577 }
578 CorruptedLogFileException t =
579 new CorruptedLogFileException("skipErrors=true Could not open hlog " +
580 path + " ignoring");
581 t.initCause(e);
582 throw t;
583 }
584 return in;
585 }
586
587 static private Entry getNextLogLine(Reader in, Path path, boolean skipErrors)
588 throws CorruptedLogFileException, IOException {
589 try {
590 return in.next();
591 } catch (EOFException eof) {
592
593 LOG.info("EOF from hlog " + path + ". continuing");
594 return null;
595 } catch (IOException e) {
596
597
598 if (e.getCause() != null &&
599 (e.getCause() instanceof ParseException ||
600 e.getCause() instanceof org.apache.hadoop.fs.ChecksumException)) {
601 LOG.warn("Parse exception " + e.getCause().toString() + " from hlog "
602 + path + ". continuing");
603 return null;
604 }
605 if (!skipErrors) {
606 throw e;
607 }
608 CorruptedLogFileException t =
609 new CorruptedLogFileException("skipErrors=true Ignoring exception" +
610 " while parsing hlog " + path + ". Marking as corrupted");
611 t.initCause(e);
612 throw t;
613 }
614 }
615
616 private void writerThreadError(Throwable t) {
617 thrown.compareAndSet(null, t);
618 }
619
620
621
622
623 private void checkForErrors() throws IOException {
624 Throwable thrown = this.thrown.get();
625 if (thrown == null) return;
626 if (thrown instanceof IOException) {
627 throw new IOException(thrown);
628 } else {
629 throw new RuntimeException(thrown);
630 }
631 }
632
633
634
635 protected Writer createWriter(FileSystem fs, Path logfile, Configuration conf)
636 throws IOException {
637 return HLogFactory.createRecoveredEditsWriter(fs, logfile, conf);
638 }
639
640
641
642
643 protected Reader getReader(FileSystem fs, Path curLogFile,
644 Configuration conf, CancelableProgressable reporter) throws IOException {
645 return HLogFactory.createReader(fs, curLogFile, conf, reporter);
646 }
647
648
649
650
651 private int getNumOpenWriters() {
652 int result = 0;
653 if (this.outputSink != null) {
654 result += this.outputSink.getNumOpenWriters();
655 }
656 return result;
657 }
658
659
660
661
662
663
664
665
666 class EntryBuffers {
667 Map<byte[], RegionEntryBuffer> buffers =
668 new TreeMap<byte[], RegionEntryBuffer>(Bytes.BYTES_COMPARATOR);
669
670
671
672
673 Set<byte[]> currentlyWriting = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
674
675 long totalBuffered = 0;
676 long maxHeapUsage;
677
678 EntryBuffers(long maxHeapUsage) {
679 this.maxHeapUsage = maxHeapUsage;
680 }
681
682
683
684
685
686
687
688
689 void appendEntry(Entry entry) throws InterruptedException, IOException {
690 HLogKey key = entry.getKey();
691
692 RegionEntryBuffer buffer;
693 long incrHeap;
694 synchronized (this) {
695 buffer = buffers.get(key.getEncodedRegionName());
696 if (buffer == null) {
697 buffer = new RegionEntryBuffer(key.getTablename(), key.getEncodedRegionName());
698 buffers.put(key.getEncodedRegionName(), buffer);
699 }
700 incrHeap= buffer.appendEntry(entry);
701 }
702
703
704 synchronized (dataAvailable) {
705 totalBuffered += incrHeap;
706 while (totalBuffered > maxHeapUsage && thrown.get() == null) {
707 LOG.debug("Used " + totalBuffered + " bytes of buffered edits, waiting for IO threads...");
708 dataAvailable.wait(2000);
709 }
710 dataAvailable.notifyAll();
711 }
712 checkForErrors();
713 }
714
715
716
717
718 synchronized RegionEntryBuffer getChunkToWrite() {
719 long biggestSize = 0;
720 byte[] biggestBufferKey = null;
721
722 for (Map.Entry<byte[], RegionEntryBuffer> entry : buffers.entrySet()) {
723 long size = entry.getValue().heapSize();
724 if (size > biggestSize && (!currentlyWriting.contains(entry.getKey()))) {
725 biggestSize = size;
726 biggestBufferKey = entry.getKey();
727 }
728 }
729 if (biggestBufferKey == null) {
730 return null;
731 }
732
733 RegionEntryBuffer buffer = buffers.remove(biggestBufferKey);
734 currentlyWriting.add(biggestBufferKey);
735 return buffer;
736 }
737
738 void doneWriting(RegionEntryBuffer buffer) {
739 synchronized (this) {
740 boolean removed = currentlyWriting.remove(buffer.encodedRegionName);
741 assert removed;
742 }
743 long size = buffer.heapSize();
744
745 synchronized (dataAvailable) {
746 totalBuffered -= size;
747
748 dataAvailable.notifyAll();
749 }
750 }
751
752 synchronized boolean isRegionCurrentlyWriting(byte[] region) {
753 return currentlyWriting.contains(region);
754 }
755 }
756
757
758
759
760
761
762
763 static class RegionEntryBuffer implements HeapSize {
764 long heapInBuffer = 0;
765 List<Entry> entryBuffer;
766 TableName tableName;
767 byte[] encodedRegionName;
768
769 RegionEntryBuffer(TableName tableName, byte[] region) {
770 this.tableName = tableName;
771 this.encodedRegionName = region;
772 this.entryBuffer = new LinkedList<Entry>();
773 }
774
775 long appendEntry(Entry entry) {
776 internify(entry);
777 entryBuffer.add(entry);
778 long incrHeap = entry.getEdit().heapSize() +
779 ClassSize.align(2 * ClassSize.REFERENCE) +
780 0;
781 heapInBuffer += incrHeap;
782 return incrHeap;
783 }
784
785 private void internify(Entry entry) {
786 HLogKey k = entry.getKey();
787 k.internTableName(this.tableName);
788 k.internEncodedRegionName(this.encodedRegionName);
789 }
790
791 public long heapSize() {
792 return heapInBuffer;
793 }
794 }
795
796 class WriterThread extends Thread {
797 private volatile boolean shouldStop = false;
798 private OutputSink outputSink = null;
799
800 WriterThread(OutputSink sink, int i) {
801 super(Thread.currentThread().getName() + "-Writer-" + i);
802 outputSink = sink;
803 }
804
805 public void run() {
806 try {
807 doRun();
808 } catch (Throwable t) {
809 LOG.error("Exiting thread", t);
810 writerThreadError(t);
811 }
812 }
813
814 private void doRun() throws IOException {
815 LOG.debug("Writer thread " + this + ": starting");
816 while (true) {
817 RegionEntryBuffer buffer = entryBuffers.getChunkToWrite();
818 if (buffer == null) {
819
820 synchronized (dataAvailable) {
821 if (shouldStop && !this.outputSink.flush()) {
822 return;
823 }
824 try {
825 dataAvailable.wait(500);
826 } catch (InterruptedException ie) {
827 if (!shouldStop) {
828 throw new RuntimeException(ie);
829 }
830 }
831 }
832 continue;
833 }
834
835 assert buffer != null;
836 try {
837 writeBuffer(buffer);
838 } finally {
839 entryBuffers.doneWriting(buffer);
840 }
841 }
842 }
843
844 private void writeBuffer(RegionEntryBuffer buffer) throws IOException {
845 outputSink.append(buffer);
846 }
847
848 void finish() {
849 synchronized (dataAvailable) {
850 shouldStop = true;
851 dataAvailable.notifyAll();
852 }
853 }
854 }
855
856
857
858
859
860 abstract class OutputSink {
861
862 protected Map<byte[], SinkWriter> writers = Collections
863 .synchronizedMap(new TreeMap<byte[], SinkWriter>(Bytes.BYTES_COMPARATOR));;
864
865 protected final Map<byte[], Long> regionMaximumEditLogSeqNum = Collections
866 .synchronizedMap(new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR));
867
868 protected final List<WriterThread> writerThreads = Lists.newArrayList();
869
870
871 protected final Set<byte[]> blacklistedRegions = Collections
872 .synchronizedSet(new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR));
873
874 protected boolean closeAndCleanCompleted = false;
875
876 protected boolean writersClosed = false;
877
878 protected final int numThreads;
879
880 protected CancelableProgressable reporter = null;
881
882 protected AtomicLong skippedEdits = new AtomicLong();
883
884 protected List<Path> splits = null;
885
886 public OutputSink(int numWriters) {
887 numThreads = numWriters;
888 }
889
890 void setReporter(CancelableProgressable reporter) {
891 this.reporter = reporter;
892 }
893
894
895
896
897 synchronized void startWriterThreads() {
898 for (int i = 0; i < numThreads; i++) {
899 WriterThread t = new WriterThread(this, i);
900 t.start();
901 writerThreads.add(t);
902 }
903 }
904
905
906
907
908
909 void updateRegionMaximumEditLogSeqNum(Entry entry) {
910 synchronized (regionMaximumEditLogSeqNum) {
911 Long currentMaxSeqNum = regionMaximumEditLogSeqNum.get(entry.getKey()
912 .getEncodedRegionName());
913 if (currentMaxSeqNum == null || entry.getKey().getLogSeqNum() > currentMaxSeqNum) {
914 regionMaximumEditLogSeqNum.put(entry.getKey().getEncodedRegionName(), entry.getKey()
915 .getLogSeqNum());
916 }
917 }
918 }
919
920 Long getRegionMaximumEditLogSeqNum(byte[] region) {
921 return regionMaximumEditLogSeqNum.get(region);
922 }
923
924
925
926
927 int getNumOpenWriters() {
928 return this.writers.size();
929 }
930
931 long getSkippedEdits() {
932 return this.skippedEdits.get();
933 }
934
935
936
937
938
939
940 protected boolean finishWriting() throws IOException {
941 LOG.info("Waiting for split writer threads to finish");
942 boolean progress_failed = false;
943 for (WriterThread t : writerThreads) {
944 t.finish();
945 }
946 for (WriterThread t : writerThreads) {
947 if (!progress_failed && reporter != null && !reporter.progress()) {
948 progress_failed = true;
949 }
950 try {
951 t.join();
952 } catch (InterruptedException ie) {
953 IOException iie = new InterruptedIOException();
954 iie.initCause(ie);
955 throw iie;
956 }
957 }
958 checkForErrors();
959 LOG.info("Split writers finished");
960 return (!progress_failed);
961 }
962
963 abstract List<Path> finishWritingAndClose() throws IOException;
964
965
966
967
968 abstract Map<byte[], Long> getOutputCounts();
969
970
971
972
973 abstract int getNumberOfRecoveredRegions();
974
975
976
977
978
979 abstract void append(RegionEntryBuffer buffer) throws IOException;
980
981
982
983
984
985 protected boolean flush() throws IOException {
986 return false;
987 }
988 }
989
990
991
992
993 class LogRecoveredEditsOutputSink extends OutputSink {
994
995 public LogRecoveredEditsOutputSink(int numWriters) {
996
997
998
999
1000
1001 super(numWriters);
1002 }
1003
1004
1005
1006
1007
1008 @Override
1009 List<Path> finishWritingAndClose() throws IOException {
1010 boolean isSuccessful = false;
1011 List<Path> result = null;
1012 try {
1013 isSuccessful = finishWriting();
1014 } finally {
1015 result = close();
1016 List<IOException> thrown = closeLogWriters(null);
1017 if (thrown != null && !thrown.isEmpty()) {
1018 throw MultipleIOException.createIOException(thrown);
1019 }
1020 }
1021 if (isSuccessful) {
1022 splits = result;
1023 }
1024 return splits;
1025 }
1026
1027
1028
1029
1030
1031 private List<Path> close() throws IOException {
1032 Preconditions.checkState(!closeAndCleanCompleted);
1033
1034 final List<Path> paths = new ArrayList<Path>();
1035 final List<IOException> thrown = Lists.newArrayList();
1036 ThreadPoolExecutor closeThreadPool = Threads.getBoundedCachedThreadPool(numThreads, 30L,
1037 TimeUnit.SECONDS, new ThreadFactory() {
1038 private int count = 1;
1039
1040 public Thread newThread(Runnable r) {
1041 Thread t = new Thread(r, "split-log-closeStream-" + count++);
1042 return t;
1043 }
1044 });
1045 CompletionService<Void> completionService =
1046 new ExecutorCompletionService<Void>(closeThreadPool);
1047 for (final Map.Entry<byte[], ? extends SinkWriter> writersEntry : writers.entrySet()) {
1048 LOG.debug("Submitting close of " + ((WriterAndPath)writersEntry.getValue()).p);
1049 completionService.submit(new Callable<Void>() {
1050 public Void call() throws Exception {
1051 WriterAndPath wap = (WriterAndPath) writersEntry.getValue();
1052 LOG.debug("Closing " + wap.p);
1053 try {
1054 wap.w.close();
1055 } catch (IOException ioe) {
1056 LOG.error("Couldn't close log at " + wap.p, ioe);
1057 thrown.add(ioe);
1058 return null;
1059 }
1060 LOG.info("Closed wap " + wap.p + " (wrote " + wap.editsWritten + " edits in "
1061 + (wap.nanosSpent / 1000 / 1000) + "ms)");
1062
1063 if (wap.editsWritten == 0) {
1064
1065 if (fs.exists(wap.p) && !fs.delete(wap.p, false)) {
1066 LOG.warn("Failed deleting empty " + wap.p);
1067 throw new IOException("Failed deleting empty " + wap.p);
1068 }
1069 return null;
1070 }
1071
1072 Path dst = getCompletedRecoveredEditsFilePath(wap.p,
1073 regionMaximumEditLogSeqNum.get(writersEntry.getKey()));
1074 try {
1075 if (!dst.equals(wap.p) && fs.exists(dst)) {
1076 LOG.warn("Found existing old edits file. It could be the "
1077 + "result of a previous failed split attempt. Deleting " + dst + ", length="
1078 + fs.getFileStatus(dst).getLen());
1079 if (!fs.delete(dst, false)) {
1080 LOG.warn("Failed deleting of old " + dst);
1081 throw new IOException("Failed deleting of old " + dst);
1082 }
1083 }
1084
1085
1086
1087 if (fs.exists(wap.p)) {
1088 if (!fs.rename(wap.p, dst)) {
1089 throw new IOException("Failed renaming " + wap.p + " to " + dst);
1090 }
1091 LOG.debug("Rename " + wap.p + " to " + dst);
1092 }
1093 } catch (IOException ioe) {
1094 LOG.error("Couldn't rename " + wap.p + " to " + dst, ioe);
1095 thrown.add(ioe);
1096 return null;
1097 }
1098 paths.add(dst);
1099 return null;
1100 }
1101 });
1102 }
1103
1104 boolean progress_failed = false;
1105 try {
1106 for (int i = 0, n = this.writers.size(); i < n; i++) {
1107 Future<Void> future = completionService.take();
1108 future.get();
1109 if (!progress_failed && reporter != null && !reporter.progress()) {
1110 progress_failed = true;
1111 }
1112 }
1113 } catch (InterruptedException e) {
1114 IOException iie = new InterruptedIOException();
1115 iie.initCause(e);
1116 throw iie;
1117 } catch (ExecutionException e) {
1118 throw new IOException(e.getCause());
1119 } finally {
1120 closeThreadPool.shutdownNow();
1121 }
1122
1123 if (!thrown.isEmpty()) {
1124 throw MultipleIOException.createIOException(thrown);
1125 }
1126 writersClosed = true;
1127 closeAndCleanCompleted = true;
1128 if (progress_failed) {
1129 return null;
1130 }
1131 return paths;
1132 }
1133
1134 private List<IOException> closeLogWriters(List<IOException> thrown) throws IOException {
1135 if (writersClosed) {
1136 return thrown;
1137 }
1138
1139 if (thrown == null) {
1140 thrown = Lists.newArrayList();
1141 }
1142 try {
1143 for (WriterThread t : writerThreads) {
1144 while (t.isAlive()) {
1145 t.shouldStop = true;
1146 t.interrupt();
1147 try {
1148 t.join(10);
1149 } catch (InterruptedException e) {
1150 IOException iie = new InterruptedIOException();
1151 iie.initCause(e);
1152 throw iie;
1153 }
1154 }
1155 }
1156 } finally {
1157 synchronized (writers) {
1158 WriterAndPath wap = null;
1159 for (SinkWriter tmpWAP : writers.values()) {
1160 try {
1161 wap = (WriterAndPath) tmpWAP;
1162 wap.w.close();
1163 } catch (IOException ioe) {
1164 LOG.error("Couldn't close log at " + wap.p, ioe);
1165 thrown.add(ioe);
1166 continue;
1167 }
1168 LOG.info("Closed log " + wap.p + " (wrote " + wap.editsWritten + " edits in "
1169 + (wap.nanosSpent / 1000 / 1000) + "ms)");
1170 }
1171 }
1172 writersClosed = true;
1173 }
1174
1175 return thrown;
1176 }
1177
1178
1179
1180
1181
1182
1183 private WriterAndPath getWriterAndPath(Entry entry) throws IOException {
1184 byte region[] = entry.getKey().getEncodedRegionName();
1185 WriterAndPath ret = (WriterAndPath) writers.get(region);
1186 if (ret != null) {
1187 return ret;
1188 }
1189
1190
1191 if (blacklistedRegions.contains(region)) {
1192 return null;
1193 }
1194 ret = createWAP(region, entry, rootDir, fs, conf);
1195 if (ret == null) {
1196 blacklistedRegions.add(region);
1197 return null;
1198 }
1199 writers.put(region, ret);
1200 return ret;
1201 }
1202
1203 private WriterAndPath createWAP(byte[] region, Entry entry, Path rootdir, FileSystem fs,
1204 Configuration conf) throws IOException {
1205 Path regionedits = getRegionSplitEditsPath(fs, entry, rootdir, true);
1206 if (regionedits == null) {
1207 return null;
1208 }
1209 if (fs.exists(regionedits)) {
1210 LOG.warn("Found old edits file. It could be the "
1211 + "result of a previous failed split attempt. Deleting " + regionedits + ", length="
1212 + fs.getFileStatus(regionedits).getLen());
1213 if (!fs.delete(regionedits, false)) {
1214 LOG.warn("Failed delete of old " + regionedits);
1215 }
1216 }
1217 Writer w = createWriter(fs, regionedits, conf);
1218 LOG.debug("Creating writer path=" + regionedits + " region=" + Bytes.toStringBinary(region));
1219 return (new WriterAndPath(regionedits, w));
1220 }
1221
1222 void append(RegionEntryBuffer buffer) throws IOException {
1223 List<Entry> entries = buffer.entryBuffer;
1224 if (entries.isEmpty()) {
1225 LOG.warn("got an empty buffer, skipping");
1226 return;
1227 }
1228
1229 WriterAndPath wap = null;
1230
1231 long startTime = System.nanoTime();
1232 try {
1233 int editsCount = 0;
1234
1235 for (Entry logEntry : entries) {
1236 if (wap == null) {
1237 wap = getWriterAndPath(logEntry);
1238 if (wap == null) {
1239
1240 return;
1241 }
1242 }
1243 wap.w.append(logEntry);
1244 this.updateRegionMaximumEditLogSeqNum(logEntry);
1245 editsCount++;
1246 }
1247
1248 wap.incrementEdits(editsCount);
1249 wap.incrementNanoTime(System.nanoTime() - startTime);
1250 } catch (IOException e) {
1251 e = RemoteExceptionHandler.checkIOException(e);
1252 LOG.fatal(" Got while writing log entry to log", e);
1253 throw e;
1254 }
1255 }
1256
1257
1258
1259
1260 Map<byte[], Long> getOutputCounts() {
1261 TreeMap<byte[], Long> ret = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
1262 synchronized (writers) {
1263 for (Map.Entry<byte[], ? extends SinkWriter> entry : writers.entrySet()) {
1264 ret.put(entry.getKey(), entry.getValue().editsWritten);
1265 }
1266 }
1267 return ret;
1268 }
1269
1270 @Override
1271 int getNumberOfRecoveredRegions() {
1272 return writers.size();
1273 }
1274 }
1275
1276
1277
1278
1279 private abstract static class SinkWriter {
1280
1281 long editsWritten = 0;
1282
1283 long nanosSpent = 0;
1284
1285 void incrementEdits(int edits) {
1286 editsWritten += edits;
1287 }
1288
1289 void incrementNanoTime(long nanos) {
1290 nanosSpent += nanos;
1291 }
1292 }
1293
1294
1295
1296
1297
1298 private final static class WriterAndPath extends SinkWriter {
1299 final Path p;
1300 final Writer w;
1301
1302 WriterAndPath(final Path p, final Writer w) {
1303 this.p = p;
1304 this.w = w;
1305 }
1306 }
1307
1308
1309
1310
1311 class LogReplayOutputSink extends OutputSink {
1312 private static final double BUFFER_THRESHOLD = 0.35;
1313 private static final String KEY_DELIMITER = "#";
1314
1315 private long waitRegionOnlineTimeOut;
1316 private final Set<String> recoveredRegions = Collections.synchronizedSet(new HashSet<String>());
1317 private final Map<String, RegionServerWriter> writers =
1318 new ConcurrentHashMap<String, RegionServerWriter>();
1319
1320 private final Map<String, HRegionLocation> onlineRegions =
1321 new ConcurrentHashMap<String, HRegionLocation>();
1322
1323 private Map<TableName, HConnection> tableNameToHConnectionMap = Collections
1324 .synchronizedMap(new TreeMap<TableName, HConnection>());
1325
1326
1327
1328
1329 private Map<String, List<Pair<HRegionLocation, HLog.Entry>>> serverToBufferQueueMap =
1330 new ConcurrentHashMap<String, List<Pair<HRegionLocation, HLog.Entry>>>();
1331 private List<Throwable> thrown = new ArrayList<Throwable>();
1332
1333
1334
1335
1336
1337 private LogRecoveredEditsOutputSink logRecoveredEditsOutputSink;
1338 private boolean hasEditsInDisablingOrDisabledTables = false;
1339
1340 public LogReplayOutputSink(int numWriters) {
1341 super(numWriters);
1342 this.waitRegionOnlineTimeOut = conf.getInt("hbase.splitlog.manager.timeout",
1343 SplitLogManager.DEFAULT_TIMEOUT);
1344 this.logRecoveredEditsOutputSink = new LogRecoveredEditsOutputSink(numWriters);
1345 this.logRecoveredEditsOutputSink.setReporter(reporter);
1346 }
1347
1348 void append(RegionEntryBuffer buffer) throws IOException {
1349 List<Entry> entries = buffer.entryBuffer;
1350 if (entries.isEmpty()) {
1351 LOG.warn("got an empty buffer, skipping");
1352 return;
1353 }
1354
1355
1356 if (disablingOrDisabledTables.contains(buffer.tableName)) {
1357
1358 logRecoveredEditsOutputSink.append(buffer);
1359 hasEditsInDisablingOrDisabledTables = true;
1360
1361 addToRecoveredRegions(Bytes.toString(buffer.encodedRegionName));
1362 return;
1363 }
1364
1365
1366 groupEditsByServer(entries);
1367
1368
1369 String maxLocKey = null;
1370 int maxSize = 0;
1371 List<Pair<HRegionLocation, HLog.Entry>> maxQueue = null;
1372 synchronized (this.serverToBufferQueueMap) {
1373 for (String key : this.serverToBufferQueueMap.keySet()) {
1374 List<Pair<HRegionLocation, HLog.Entry>> curQueue = this.serverToBufferQueueMap.get(key);
1375 if (curQueue.size() > maxSize) {
1376 maxSize = curQueue.size();
1377 maxQueue = curQueue;
1378 maxLocKey = key;
1379 }
1380 }
1381 if (maxSize < minBatchSize
1382 && entryBuffers.totalBuffered < BUFFER_THRESHOLD * entryBuffers.maxHeapUsage) {
1383
1384 return;
1385 } else if (maxSize > 0) {
1386 this.serverToBufferQueueMap.remove(maxLocKey);
1387 }
1388 }
1389
1390 if (maxSize > 0) {
1391 processWorkItems(maxLocKey, maxQueue);
1392 }
1393 }
1394
1395 private void addToRecoveredRegions(String encodedRegionName) {
1396 if (!recoveredRegions.contains(encodedRegionName)) {
1397 recoveredRegions.add(encodedRegionName);
1398 }
1399 }
1400
1401
1402
1403
1404
1405 private void groupEditsByServer(List<Entry> entries) throws IOException {
1406 Set<TableName> nonExistentTables = null;
1407 Long cachedLastFlushedSequenceId = -1l;
1408 for (HLog.Entry entry : entries) {
1409 WALEdit edit = entry.getEdit();
1410 TableName table = entry.getKey().getTablename();
1411
1412 entry.getKey().setScopes(null);
1413 String encodeRegionNameStr = Bytes.toString(entry.getKey().getEncodedRegionName());
1414
1415 if (nonExistentTables != null && nonExistentTables.contains(table)) {
1416 this.skippedEdits.incrementAndGet();
1417 continue;
1418 }
1419
1420 Map<byte[], Long> maxStoreSequenceIds = null;
1421 boolean needSkip = false;
1422 HRegionLocation loc = null;
1423 String locKey = null;
1424 List<KeyValue> kvs = edit.getKeyValues();
1425 List<KeyValue> skippedKVs = new ArrayList<KeyValue>();
1426 HConnection hconn = this.getConnectionByTableName(table);
1427
1428 for (KeyValue kv : kvs) {
1429
1430
1431
1432
1433 if (kv.matchingFamily(WALEdit.METAFAMILY)) {
1434 skippedKVs.add(kv);
1435 continue;
1436 }
1437
1438 try {
1439 loc =
1440 locateRegionAndRefreshLastFlushedSequenceId(hconn, table, kv.getRow(),
1441 encodeRegionNameStr);
1442 } catch (TableNotFoundException ex) {
1443
1444 LOG.info("Table " + table + " doesn't exist. Skip log replay for region "
1445 + encodeRegionNameStr);
1446 lastFlushedSequenceIds.put(encodeRegionNameStr, Long.MAX_VALUE);
1447 if (nonExistentTables == null) {
1448 nonExistentTables = new TreeSet<TableName>();
1449 }
1450 nonExistentTables.add(table);
1451 this.skippedEdits.incrementAndGet();
1452 needSkip = true;
1453 break;
1454 }
1455
1456 cachedLastFlushedSequenceId =
1457 lastFlushedSequenceIds.get(loc.getRegionInfo().getEncodedName());
1458 if (cachedLastFlushedSequenceId != null
1459 && cachedLastFlushedSequenceId >= entry.getKey().getLogSeqNum()) {
1460
1461 this.skippedEdits.incrementAndGet();
1462 needSkip = true;
1463 break;
1464 } else {
1465 if (maxStoreSequenceIds == null) {
1466 maxStoreSequenceIds =
1467 regionMaxSeqIdInStores.get(loc.getRegionInfo().getEncodedName());
1468 }
1469 if (maxStoreSequenceIds != null) {
1470 Long maxStoreSeqId = maxStoreSequenceIds.get(kv.getFamily());
1471 if (maxStoreSeqId == null || maxStoreSeqId >= entry.getKey().getLogSeqNum()) {
1472
1473 skippedKVs.add(kv);
1474 continue;
1475 }
1476 }
1477 }
1478 }
1479
1480
1481 if (loc == null || needSkip) continue;
1482
1483 if (!skippedKVs.isEmpty()) {
1484 kvs.removeAll(skippedKVs);
1485 }
1486 synchronized (serverToBufferQueueMap) {
1487 locKey = loc.getHostnamePort() + KEY_DELIMITER + table;
1488 List<Pair<HRegionLocation, HLog.Entry>> queue = serverToBufferQueueMap.get(locKey);
1489 if (queue == null) {
1490 queue =
1491 Collections.synchronizedList(new ArrayList<Pair<HRegionLocation, HLog.Entry>>());
1492 serverToBufferQueueMap.put(locKey, queue);
1493 }
1494 queue.add(new Pair<HRegionLocation, HLog.Entry>(loc, entry));
1495 }
1496
1497 addToRecoveredRegions(loc.getRegionInfo().getEncodedName());
1498 }
1499 }
1500
1501
1502
1503
1504
1505
1506 private HRegionLocation locateRegionAndRefreshLastFlushedSequenceId(HConnection hconn,
1507 TableName table, byte[] row, String originalEncodedRegionName) throws IOException {
1508
1509 HRegionLocation loc = onlineRegions.get(originalEncodedRegionName);
1510 if(loc != null) return loc;
1511
1512 loc = hconn.getRegionLocation(table, row, true);
1513 if (loc == null) {
1514 throw new IOException("Can't locate location for row:" + Bytes.toString(row)
1515 + " of table:" + table);
1516 }
1517
1518 if (!originalEncodedRegionName.equalsIgnoreCase(loc.getRegionInfo().getEncodedName())) {
1519
1520 lastFlushedSequenceIds.put(originalEncodedRegionName, Long.MAX_VALUE);
1521 HRegionLocation tmpLoc = onlineRegions.get(loc.getRegionInfo().getEncodedName());
1522 if (tmpLoc != null) return tmpLoc;
1523 }
1524
1525 Long lastFlushedSequenceId = -1l;
1526 AtomicBoolean isRecovering = new AtomicBoolean(true);
1527 loc = waitUntilRegionOnline(loc, row, this.waitRegionOnlineTimeOut, isRecovering);
1528 if (!isRecovering.get()) {
1529
1530
1531 lastFlushedSequenceIds.put(loc.getRegionInfo().getEncodedName(), Long.MAX_VALUE);
1532 LOG.info("logReplay skip region: " + loc.getRegionInfo().getEncodedName()
1533 + " because it's not in recovering.");
1534 } else {
1535 Long cachedLastFlushedSequenceId =
1536 lastFlushedSequenceIds.get(loc.getRegionInfo().getEncodedName());
1537
1538
1539
1540 RegionStoreSequenceIds ids =
1541 SplitLogManager.getRegionFlushedSequenceId(watcher, failedServerName, loc
1542 .getRegionInfo().getEncodedName());
1543 if (ids != null) {
1544 lastFlushedSequenceId = ids.getLastFlushedSequenceId();
1545 Map<byte[], Long> storeIds = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
1546 List<StoreSequenceId> maxSeqIdInStores = ids.getStoreSequenceIdList();
1547 for (StoreSequenceId id : maxSeqIdInStores) {
1548 storeIds.put(id.getFamilyName().toByteArray(), id.getSequenceId());
1549 }
1550 regionMaxSeqIdInStores.put(loc.getRegionInfo().getEncodedName(), storeIds);
1551 }
1552
1553 if (cachedLastFlushedSequenceId == null
1554 || lastFlushedSequenceId > cachedLastFlushedSequenceId) {
1555 lastFlushedSequenceIds.put(loc.getRegionInfo().getEncodedName(), lastFlushedSequenceId);
1556 }
1557 }
1558
1559 onlineRegions.put(loc.getRegionInfo().getEncodedName(), loc);
1560 return loc;
1561 }
1562
1563 private void processWorkItems(String key, List<Pair<HRegionLocation, HLog.Entry>> actions)
1564 throws IOException {
1565 RegionServerWriter rsw = null;
1566
1567 long startTime = System.nanoTime();
1568 try {
1569 rsw = getRegionServerWriter(key);
1570 rsw.sink.replayEntries(actions);
1571
1572
1573 rsw.incrementEdits(actions.size());
1574 rsw.incrementNanoTime(System.nanoTime() - startTime);
1575 } catch (IOException e) {
1576 e = RemoteExceptionHandler.checkIOException(e);
1577 LOG.fatal(" Got while writing log entry to log", e);
1578 throw e;
1579 }
1580 }
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591 private HRegionLocation waitUntilRegionOnline(HRegionLocation loc, byte[] row,
1592 final long timeout, AtomicBoolean isRecovering)
1593 throws IOException {
1594 final long endTime = EnvironmentEdgeManager.currentTimeMillis() + timeout;
1595 final long pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
1596 HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
1597 boolean reloadLocation = false;
1598 TableName tableName = loc.getRegionInfo().getTable();
1599 int tries = 0;
1600 Throwable cause = null;
1601 while (endTime > EnvironmentEdgeManager.currentTimeMillis()) {
1602 try {
1603
1604 HConnection hconn = getConnectionByTableName(tableName);
1605 if(reloadLocation) {
1606 loc = hconn.getRegionLocation(tableName, row, true);
1607 }
1608 BlockingInterface remoteSvr = hconn.getAdmin(loc.getServerName());
1609 HRegionInfo region = loc.getRegionInfo();
1610 try {
1611 GetRegionInfoRequest request =
1612 RequestConverter.buildGetRegionInfoRequest(region.getRegionName());
1613 GetRegionInfoResponse response = remoteSvr.getRegionInfo(null, request);
1614 if (HRegionInfo.convert(response.getRegionInfo()) != null) {
1615 isRecovering.set((response.hasIsRecovering()) ? response.getIsRecovering() : true);
1616 return loc;
1617 }
1618 } catch (ServiceException se) {
1619 throw ProtobufUtil.getRemoteException(se);
1620 }
1621 } catch (IOException e) {
1622 cause = e.getCause();
1623 if(!(cause instanceof RegionOpeningException)) {
1624 reloadLocation = true;
1625 }
1626 }
1627 long expectedSleep = ConnectionUtils.getPauseTime(pause, tries);
1628 try {
1629 Thread.sleep(expectedSleep);
1630 } catch (InterruptedException e) {
1631 Thread.currentThread().interrupt();
1632 throw new IOException("Interrupted when waiting regon " +
1633 loc.getRegionInfo().getEncodedName() + " online.", e);
1634 }
1635 tries++;
1636 }
1637
1638 throw new IOException("Timeout when waiting region " + loc.getRegionInfo().getEncodedName() +
1639 " online for " + timeout + " milliseconds.", cause);
1640 }
1641
1642 @Override
1643 protected boolean flush() throws IOException {
1644 String curLoc = null;
1645 int curSize = 0;
1646 List<Pair<HRegionLocation, HLog.Entry>> curQueue = null;
1647 synchronized (this.serverToBufferQueueMap) {
1648 for (String locationKey : this.serverToBufferQueueMap.keySet()) {
1649 curQueue = this.serverToBufferQueueMap.get(locationKey);
1650 if (!curQueue.isEmpty()) {
1651 curSize = curQueue.size();
1652 curLoc = locationKey;
1653 break;
1654 }
1655 }
1656 if (curSize > 0) {
1657 this.serverToBufferQueueMap.remove(curLoc);
1658 }
1659 }
1660
1661 if (curSize > 0) {
1662 this.processWorkItems(curLoc, curQueue);
1663 dataAvailable.notifyAll();
1664 return true;
1665 }
1666 return false;
1667 }
1668
1669 void addWriterError(Throwable t) {
1670 thrown.add(t);
1671 }
1672
1673 @Override
1674 List<Path> finishWritingAndClose() throws IOException {
1675 try {
1676 if (!finishWriting()) {
1677 return null;
1678 }
1679 if (hasEditsInDisablingOrDisabledTables) {
1680 splits = logRecoveredEditsOutputSink.finishWritingAndClose();
1681 } else {
1682 splits = new ArrayList<Path>();
1683 }
1684
1685 return splits;
1686 } finally {
1687 List<IOException> thrown = closeRegionServerWriters();
1688 if (thrown != null && !thrown.isEmpty()) {
1689 throw MultipleIOException.createIOException(thrown);
1690 }
1691 }
1692 }
1693
1694 @Override
1695 int getNumOpenWriters() {
1696 return this.writers.size() + this.logRecoveredEditsOutputSink.getNumOpenWriters();
1697 }
1698
1699 private List<IOException> closeRegionServerWriters() throws IOException {
1700 List<IOException> result = null;
1701 if (!writersClosed) {
1702 result = Lists.newArrayList();
1703 try {
1704 for (WriterThread t : writerThreads) {
1705 while (t.isAlive()) {
1706 t.shouldStop = true;
1707 t.interrupt();
1708 try {
1709 t.join(10);
1710 } catch (InterruptedException e) {
1711 IOException iie = new InterruptedIOException();
1712 iie.initCause(e);
1713 throw iie;
1714 }
1715 }
1716 }
1717 } finally {
1718 synchronized (writers) {
1719 for (String locationKey : writers.keySet()) {
1720 RegionServerWriter tmpW = writers.get(locationKey);
1721 try {
1722 tmpW.close();
1723 } catch (IOException ioe) {
1724 LOG.error("Couldn't close writer for region server:" + locationKey, ioe);
1725 result.add(ioe);
1726 }
1727 }
1728 }
1729
1730
1731 synchronized (this.tableNameToHConnectionMap) {
1732 for (TableName tableName : this.tableNameToHConnectionMap.keySet()) {
1733 HConnection hconn = this.tableNameToHConnectionMap.get(tableName);
1734 try {
1735 hconn.clearRegionCache();
1736 hconn.close();
1737 } catch (IOException ioe) {
1738 result.add(ioe);
1739 }
1740 }
1741 }
1742 writersClosed = true;
1743 }
1744 }
1745 return result;
1746 }
1747
1748 Map<byte[], Long> getOutputCounts() {
1749 TreeMap<byte[], Long> ret = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
1750 synchronized (writers) {
1751 for (Map.Entry<String, RegionServerWriter> entry : writers.entrySet()) {
1752 ret.put(Bytes.toBytes(entry.getKey()), entry.getValue().editsWritten);
1753 }
1754 }
1755 return ret;
1756 }
1757
1758 @Override
1759 int getNumberOfRecoveredRegions() {
1760 return this.recoveredRegions.size();
1761 }
1762
1763
1764
1765
1766
1767
1768 private RegionServerWriter getRegionServerWriter(String loc) throws IOException {
1769 RegionServerWriter ret = writers.get(loc);
1770 if (ret != null) {
1771 return ret;
1772 }
1773
1774 TableName tableName = getTableFromLocationStr(loc);
1775 if(tableName == null){
1776 throw new IOException("Invalid location string:" + loc + " found. Replay aborted.");
1777 }
1778
1779 HConnection hconn = getConnectionByTableName(tableName);
1780 synchronized (writers) {
1781 ret = writers.get(loc);
1782 if (ret == null) {
1783 ret = new RegionServerWriter(conf, tableName, hconn);
1784 writers.put(loc, ret);
1785 }
1786 }
1787 return ret;
1788 }
1789
1790 private HConnection getConnectionByTableName(final TableName tableName) throws IOException {
1791 HConnection hconn = this.tableNameToHConnectionMap.get(tableName);
1792 if (hconn == null) {
1793 synchronized (this.tableNameToHConnectionMap) {
1794 hconn = this.tableNameToHConnectionMap.get(tableName);
1795 if (hconn == null) {
1796 hconn = HConnectionManager.getConnection(conf);
1797 this.tableNameToHConnectionMap.put(tableName, hconn);
1798 }
1799 }
1800 }
1801 return hconn;
1802 }
1803 private TableName getTableFromLocationStr(String loc) {
1804
1805
1806
1807 String[] splits = loc.split(KEY_DELIMITER);
1808 if (splits.length != 2) {
1809 return null;
1810 }
1811 return TableName.valueOf(splits[1]);
1812 }
1813 }
1814
1815
1816
1817
1818
1819 private final static class RegionServerWriter extends SinkWriter {
1820 final WALEditsReplaySink sink;
1821
1822 RegionServerWriter(final Configuration conf, final TableName tableName, final HConnection conn)
1823 throws IOException {
1824 this.sink = new WALEditsReplaySink(conf, tableName, conn);
1825 }
1826
1827 void close() throws IOException {
1828 }
1829 }
1830
1831 static class CorruptedLogFileException extends Exception {
1832 private static final long serialVersionUID = 1L;
1833
1834 CorruptedLogFileException(String s) {
1835 super(s);
1836 }
1837 }
1838
1839
1840
1841
1842
1843
1844
1845
1846
1847
1848
1849 public static List<Pair<MutationType, Mutation>> getMutationsFromWALEntry(WALEntry entry,
1850 CellScanner cells, Pair<HLogKey, WALEdit> logEntry) throws IOException {
1851
1852 if (entry == null) {
1853
1854 return new ArrayList<Pair<MutationType, Mutation>>();
1855 }
1856
1857 int count = entry.getAssociatedCellCount();
1858 List<Pair<MutationType, Mutation>> mutations = new ArrayList<Pair<MutationType, Mutation>>();
1859 Cell previousCell = null;
1860 Mutation m = null;
1861 HLogKey key = null;
1862 WALEdit val = null;
1863 if (logEntry != null) val = new WALEdit();
1864
1865 for (int i = 0; i < count; i++) {
1866
1867 if (!cells.advance()) {
1868 throw new ArrayIndexOutOfBoundsException("Expected=" + count + ", index=" + i);
1869 }
1870 Cell cell = cells.current();
1871 if (val != null) val.add(KeyValueUtil.ensureKeyValue(cell));
1872
1873 boolean isNewRowOrType =
1874 previousCell == null || previousCell.getTypeByte() != cell.getTypeByte()
1875 || !CellUtil.matchingRow(previousCell, cell);
1876 if (isNewRowOrType) {
1877
1878 if (CellUtil.isDelete(cell)) {
1879 m = new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
1880 mutations.add(new Pair<MutationType, Mutation>(MutationType.DELETE, m));
1881 } else {
1882 m = new Put(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength());
1883 mutations.add(new Pair<MutationType, Mutation>(MutationType.PUT, m));
1884 }
1885 }
1886 if (CellUtil.isDelete(cell)) {
1887 ((Delete) m).addDeleteMarker(KeyValueUtil.ensureKeyValue(cell));
1888 } else {
1889 ((Put) m).add(KeyValueUtil.ensureKeyValue(cell));
1890 }
1891 previousCell = cell;
1892 }
1893
1894
1895 if (logEntry != null) {
1896 WALKey walKey = entry.getKey();
1897 List<UUID> clusterIds = new ArrayList<UUID>(walKey.getClusterIdsCount());
1898 for (HBaseProtos.UUID uuid : entry.getKey().getClusterIdsList()) {
1899 clusterIds.add(new UUID(uuid.getMostSigBits(), uuid.getLeastSigBits()));
1900 }
1901 key = new HLogKey(walKey.getEncodedRegionName().toByteArray(), TableName.valueOf(walKey
1902 .getTableName().toByteArray()), walKey.getLogSequenceNumber(), walKey.getWriteTime(),
1903 clusterIds);
1904 logEntry.setFirst(key);
1905 logEntry.setSecond(val);
1906 }
1907
1908 return mutations;
1909 }
1910 }