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