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