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.FileNotFoundException;
22 import java.io.IOException;
23 import java.io.OutputStream;
24 import java.lang.reflect.InvocationTargetException;
25 import java.lang.reflect.Method;
26 import java.net.URLEncoder;
27 import java.util.ArrayList;
28 import java.util.Arrays;
29 import java.util.Collections;
30 import java.util.LinkedList;
31 import java.util.List;
32 import java.util.Map;
33 import java.util.SortedMap;
34 import java.util.TreeMap;
35 import java.util.TreeSet;
36 import java.util.UUID;
37 import java.util.concurrent.ConcurrentSkipListMap;
38 import java.util.concurrent.CopyOnWriteArrayList;
39 import java.util.concurrent.atomic.AtomicBoolean;
40 import java.util.concurrent.atomic.AtomicInteger;
41 import java.util.concurrent.atomic.AtomicLong;
42 import java.util.concurrent.locks.ReentrantLock;
43
44 import org.apache.commons.logging.Log;
45 import org.apache.commons.logging.LogFactory;
46 import org.apache.hadoop.classification.InterfaceAudience;
47 import org.apache.hadoop.conf.Configuration;
48 import org.apache.hadoop.fs.FSDataOutputStream;
49 import org.apache.hadoop.fs.FileStatus;
50 import org.apache.hadoop.fs.FileSystem;
51 import org.apache.hadoop.fs.Path;
52 import org.apache.hadoop.fs.Syncable;
53 import org.apache.hadoop.hbase.HBaseConfiguration;
54 import org.apache.hadoop.hbase.HConstants;
55 import org.apache.hadoop.hbase.HRegionInfo;
56 import org.apache.hadoop.hbase.HTableDescriptor;
57 import org.apache.hadoop.hbase.KeyValue;
58 import org.apache.hadoop.hbase.TableName;
59 import org.apache.hadoop.hbase.util.Bytes;
60 import org.apache.hadoop.hbase.util.ClassSize;
61 import org.apache.hadoop.hbase.util.DrainBarrier;
62 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
63 import org.apache.hadoop.hbase.util.FSUtils;
64 import org.apache.hadoop.hbase.util.HasThread;
65 import org.apache.hadoop.hbase.util.Threads;
66 import org.apache.hadoop.util.StringUtils;
67 import org.cloudera.htrace.Trace;
68 import org.cloudera.htrace.TraceScope;
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109 @InterfaceAudience.Private
110 class FSHLog implements HLog, Syncable {
111 static final Log LOG = LogFactory.getLog(FSHLog.class);
112
113 private final FileSystem fs;
114 private final Path rootDir;
115 private final Path dir;
116 private final Configuration conf;
117
118 private List<WALActionsListener> listeners =
119 new CopyOnWriteArrayList<WALActionsListener>();
120 private final long optionalFlushInterval;
121 private final long blocksize;
122 private final String prefix;
123 private final AtomicLong unflushedEntries = new AtomicLong(0);
124 private volatile long syncedTillHere = 0;
125 private long lastDeferredTxid;
126 private final Path oldLogDir;
127
128 private WALCoprocessorHost coprocessorHost;
129
130 private FSDataOutputStream hdfs_out;
131
132
133 private int minTolerableReplication;
134 private Method getNumCurrentReplicas;
135 final static Object [] NO_ARGS = new Object []{};
136
137
138 private DrainBarrier closeBarrier = new DrainBarrier();
139
140
141
142
143 Writer writer;
144
145
146
147
148 final SortedMap<Long, Path> outputfiles =
149 Collections.synchronizedSortedMap(new TreeMap<Long, Path>());
150
151
152
153
154
155
156
157 private final Object oldestSeqNumsLock = new Object();
158
159
160
161
162
163 private final ReentrantLock rollWriterLock = new ReentrantLock(true);
164
165
166
167
168 private final ConcurrentSkipListMap<byte [], Long> oldestUnflushedSeqNums =
169 new ConcurrentSkipListMap<byte [], Long>(Bytes.BYTES_COMPARATOR);
170
171
172
173
174
175 private final Map<byte[], Long> oldestFlushingSeqNums =
176 new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
177
178 private volatile boolean closed = false;
179
180 private final AtomicLong logSeqNum = new AtomicLong(0);
181
182 private boolean forMeta = false;
183
184
185 private volatile long filenum = -1;
186
187
188 private final AtomicInteger numEntries = new AtomicInteger(0);
189
190
191
192
193
194 private AtomicInteger consecutiveLogRolls = new AtomicInteger(0);
195 private final int lowReplicationRollLimit;
196
197
198
199
200 private volatile boolean lowReplicationRollEnabled = true;
201
202
203
204 private final long logrollsize;
205
206
207 private long curLogSize = 0;
208
209
210
211
212 private AtomicLong totalLogSize = new AtomicLong(0);
213
214
215
216
217 private final Object updateLock = new Object();
218 private final Object flushLock = new Object();
219
220 private final boolean enabled;
221
222
223
224
225
226
227 private final int maxLogs;
228
229
230
231
232 private final LogSyncer logSyncer;
233
234
235 private final int closeErrorsTolerated;
236
237 private final AtomicInteger closeErrorCount = new AtomicInteger();
238 private final MetricsWAL metrics;
239
240
241
242
243
244
245
246
247
248
249 public FSHLog(final FileSystem fs, final Path root, final String logDir,
250 final Configuration conf)
251 throws IOException {
252 this(fs, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME,
253 conf, null, true, null, false);
254 }
255
256
257
258
259
260
261
262
263
264
265
266 public FSHLog(final FileSystem fs, final Path root, final String logDir,
267 final String oldLogDir, final Configuration conf)
268 throws IOException {
269 this(fs, root, logDir, oldLogDir,
270 conf, null, true, null, false);
271 }
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292 public FSHLog(final FileSystem fs, final Path root, final String logDir,
293 final Configuration conf, final List<WALActionsListener> listeners,
294 final String prefix) throws IOException {
295 this(fs, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME,
296 conf, listeners, true, prefix, false);
297 }
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321 public FSHLog(final FileSystem fs, final Path root, final String logDir,
322 final String oldLogDir, final Configuration conf,
323 final List<WALActionsListener> listeners,
324 final boolean failIfLogDirExists, final String prefix, boolean forMeta)
325 throws IOException {
326 super();
327 this.fs = fs;
328 this.rootDir = root;
329 this.dir = new Path(this.rootDir, logDir);
330 this.oldLogDir = new Path(this.rootDir, oldLogDir);
331 this.forMeta = forMeta;
332 this.conf = conf;
333
334 if (listeners != null) {
335 for (WALActionsListener i: listeners) {
336 registerWALActionsListener(i);
337 }
338 }
339
340 this.blocksize = this.conf.getLong("hbase.regionserver.hlog.blocksize",
341 FSUtils.getDefaultBlockSize(this.fs, this.dir));
342
343 float multi = conf.getFloat("hbase.regionserver.logroll.multiplier", 0.95f);
344 this.logrollsize = (long)(this.blocksize * multi);
345 this.optionalFlushInterval =
346 conf.getLong("hbase.regionserver.optionallogflushinterval", 1 * 1000);
347
348 this.maxLogs = conf.getInt("hbase.regionserver.maxlogs", 32);
349 this.minTolerableReplication = conf.getInt(
350 "hbase.regionserver.hlog.tolerable.lowreplication",
351 FSUtils.getDefaultReplication(fs, this.dir));
352 this.lowReplicationRollLimit = conf.getInt(
353 "hbase.regionserver.hlog.lowreplication.rolllimit", 5);
354 this.enabled = conf.getBoolean("hbase.regionserver.hlog.enabled", true);
355 this.closeErrorsTolerated = conf.getInt(
356 "hbase.regionserver.logroll.errors.tolerated", 0);
357
358 this.logSyncer = new LogSyncer(this.optionalFlushInterval);
359
360 LOG.info("WAL/HLog configuration: blocksize=" +
361 StringUtils.byteDesc(this.blocksize) +
362 ", rollsize=" + StringUtils.byteDesc(this.logrollsize) +
363 ", enabled=" + this.enabled +
364 ", optionallogflushinternal=" + this.optionalFlushInterval + "ms");
365
366 this.prefix = prefix == null || prefix.isEmpty() ?
367 "hlog" : URLEncoder.encode(prefix, "UTF8");
368
369 boolean dirExists = false;
370 if (failIfLogDirExists && (dirExists = this.fs.exists(dir))) {
371 throw new IOException("Target HLog directory already exists: " + dir);
372 }
373 if (!dirExists && !fs.mkdirs(dir)) {
374 throw new IOException("Unable to mkdir " + dir);
375 }
376
377 if (!fs.exists(this.oldLogDir)) {
378 if (!fs.mkdirs(this.oldLogDir)) {
379 throw new IOException("Unable to mkdir " + this.oldLogDir);
380 }
381 }
382
383 rollWriter();
384
385
386 this.getNumCurrentReplicas = getGetNumCurrentReplicas(this.hdfs_out);
387
388
389 if (this.optionalFlushInterval > 0) {
390 Threads.setDaemonThreadRunning(logSyncer.getThread(), Thread.currentThread().getName()
391 + ".logSyncer");
392 } else {
393 LOG.info("hbase.regionserver.optionallogflushinterval is set as "
394 + this.optionalFlushInterval + ". Deferred log syncing won't work. "
395 + "Any Mutation, marked to be deferred synced, will be flushed immediately.");
396 }
397 coprocessorHost = new WALCoprocessorHost(this, conf);
398
399 this.metrics = new MetricsWAL();
400 }
401
402
403
404
405
406 private Method getGetNumCurrentReplicas(final FSDataOutputStream os) {
407 Method m = null;
408 if (os != null) {
409 Class<? extends OutputStream> wrappedStreamClass = os.getWrappedStream()
410 .getClass();
411 try {
412 m = wrappedStreamClass.getDeclaredMethod("getNumCurrentReplicas",
413 new Class<?>[] {});
414 m.setAccessible(true);
415 } catch (NoSuchMethodException e) {
416 LOG.info("FileSystem's output stream doesn't support"
417 + " getNumCurrentReplicas; --HDFS-826 not available; fsOut="
418 + wrappedStreamClass.getName());
419 } catch (SecurityException e) {
420 LOG.info("Doesn't have access to getNumCurrentReplicas on "
421 + "FileSystems's output stream --HDFS-826 not available; fsOut="
422 + wrappedStreamClass.getName(), e);
423 m = null;
424 }
425 }
426 if (m != null) {
427 if (LOG.isTraceEnabled()) LOG.trace("Using getNumCurrentReplicas--HDFS-826");
428 }
429 return m;
430 }
431
432 @Override
433 public void registerWALActionsListener(final WALActionsListener listener) {
434 this.listeners.add(listener);
435 }
436
437 @Override
438 public boolean unregisterWALActionsListener(final WALActionsListener listener) {
439 return this.listeners.remove(listener);
440 }
441
442 @Override
443 public long getFilenum() {
444 return this.filenum;
445 }
446
447 @Override
448 public void setSequenceNumber(final long newvalue) {
449 for (long id = this.logSeqNum.get(); id < newvalue &&
450 !this.logSeqNum.compareAndSet(id, newvalue); id = this.logSeqNum.get()) {
451
452
453 LOG.debug("Changed sequenceid from " + id + " to " + newvalue);
454 }
455 }
456
457 @Override
458 public long getSequenceNumber() {
459 return logSeqNum.get();
460 }
461
462
463
464
465
466
467
468
469
470 OutputStream getOutputStream() {
471 return this.hdfs_out.getWrappedStream();
472 }
473
474 @Override
475 public byte [][] rollWriter() throws FailedLogCloseException, IOException {
476 return rollWriter(false);
477 }
478
479 @Override
480 public byte [][] rollWriter(boolean force)
481 throws FailedLogCloseException, IOException {
482 rollWriterLock.lock();
483 try {
484
485 if (!force && this.writer != null && this.numEntries.get() <= 0) {
486 return null;
487 }
488 byte [][] regionsToFlush = null;
489 if (closed) {
490 LOG.debug("HLog closed. Skipping rolling of writer");
491 return null;
492 }
493 try {
494 if (!closeBarrier.beginOp()) {
495 LOG.debug("HLog closing. Skipping rolling of writer");
496 return regionsToFlush;
497 }
498
499
500 long currentFilenum = this.filenum;
501 Path oldPath = null;
502 if (currentFilenum > 0) {
503
504 oldPath = computeFilename(currentFilenum);
505 }
506 this.filenum = System.currentTimeMillis();
507 Path newPath = computeFilename();
508 while (fs.exists(newPath)) {
509 this.filenum++;
510 newPath = computeFilename();
511 }
512
513
514 if (!this.listeners.isEmpty()) {
515 for (WALActionsListener i : this.listeners) {
516 i.preLogRoll(oldPath, newPath);
517 }
518 }
519 FSHLog.Writer nextWriter = this.createWriterInstance(fs, newPath, conf);
520
521 FSDataOutputStream nextHdfsOut = null;
522 if (nextWriter instanceof ProtobufLogWriter) {
523 nextHdfsOut = ((ProtobufLogWriter)nextWriter).getStream();
524
525 try {
526 nextWriter.sync();
527 } catch (IOException e) {
528
529 LOG.warn("pre-sync failed", e);
530 }
531 }
532
533 Path oldFile = null;
534 int oldNumEntries = 0;
535 synchronized (updateLock) {
536
537 oldNumEntries = this.numEntries.get();
538 oldFile = cleanupCurrentWriter(currentFilenum);
539 this.writer = nextWriter;
540 this.hdfs_out = nextHdfsOut;
541 this.numEntries.set(0);
542 }
543 if (oldFile == null) LOG.info("New WAL " + FSUtils.getPath(newPath));
544 else {
545 long oldFileLen = this.fs.getFileStatus(oldFile).getLen();
546 this.totalLogSize.addAndGet(oldFileLen);
547 LOG.info("Rolled WAL " + FSUtils.getPath(oldFile) + " with entries="
548 + oldNumEntries + ", filesize="
549 + StringUtils.humanReadableInt(oldFileLen) + "; new WAL "
550 + FSUtils.getPath(newPath));
551 }
552
553
554 if (!this.listeners.isEmpty()) {
555 for (WALActionsListener i : this.listeners) {
556 i.postLogRoll(oldPath, newPath);
557 }
558 }
559
560
561 if (getNumRolledLogFiles() > 0) {
562 cleanOldLogs();
563 regionsToFlush = getRegionsToForceFlush();
564 }
565 } finally {
566 closeBarrier.endOp();
567 }
568 return regionsToFlush;
569 } finally {
570 rollWriterLock.unlock();
571 }
572 }
573
574
575
576
577
578
579
580
581
582
583
584 protected Writer createWriterInstance(final FileSystem fs, final Path path,
585 final Configuration conf) throws IOException {
586 if (forMeta) {
587
588 }
589 return HLogFactory.createWALWriter(fs, path, conf);
590 }
591
592
593
594
595
596
597
598
599 private void cleanOldLogs() throws IOException {
600 long oldestOutstandingSeqNum = Long.MAX_VALUE;
601 synchronized (oldestSeqNumsLock) {
602 Long oldestFlushing = (oldestFlushingSeqNums.size() > 0)
603 ? Collections.min(oldestFlushingSeqNums.values()) : Long.MAX_VALUE;
604 Long oldestUnflushed = (oldestUnflushedSeqNums.size() > 0)
605 ? Collections.min(oldestUnflushedSeqNums.values()) : Long.MAX_VALUE;
606 oldestOutstandingSeqNum = Math.min(oldestFlushing, oldestUnflushed);
607 }
608
609
610
611 TreeSet<Long> sequenceNumbers = new TreeSet<Long>(this.outputfiles.headMap(
612 oldestOutstandingSeqNum).keySet());
613
614 if (LOG.isDebugEnabled()) {
615 if (sequenceNumbers.size() > 0) {
616 LOG.debug("Found " + sequenceNumbers.size() + " hlogs to remove" +
617 " out of total " + this.outputfiles.size() + ";" +
618 " oldest outstanding sequenceid is " + oldestOutstandingSeqNum);
619 }
620 }
621 for (Long seq : sequenceNumbers) {
622 Path p = this.outputfiles.remove(seq);
623 if (p != null) this.totalLogSize.addAndGet(-this.fs.getFileStatus(p).getLen());
624 archiveLogFile(p, seq);
625 }
626 }
627
628
629
630
631
632
633
634
635 static byte[][] findMemstoresWithEditsEqualOrOlderThan(
636 final long walSeqNum, final Map<byte[], Long> regionsToSeqNums) {
637 List<byte[]> regions = null;
638 for (Map.Entry<byte[], Long> e : regionsToSeqNums.entrySet()) {
639 if (e.getValue().longValue() <= walSeqNum) {
640 if (regions == null) regions = new ArrayList<byte[]>();
641 regions.add(e.getKey());
642 }
643 }
644 return regions == null ? null : regions
645 .toArray(new byte[][] { HConstants.EMPTY_BYTE_ARRAY });
646 }
647
648 private byte[][] getRegionsToForceFlush() throws IOException {
649
650
651 byte [][] regions = null;
652 int logCount = getNumRolledLogFiles();
653 if (logCount > this.maxLogs && logCount > 0) {
654
655 synchronized (oldestSeqNumsLock) {
656 regions = findMemstoresWithEditsEqualOrOlderThan(this.outputfiles.firstKey(),
657 this.oldestUnflushedSeqNums);
658 }
659 if (regions != null) {
660 StringBuilder sb = new StringBuilder();
661 for (int i = 0; i < regions.length; i++) {
662 if (i > 0) sb.append(", ");
663 sb.append(Bytes.toStringBinary(regions[i]));
664 }
665 LOG.info("Too many hlogs: logs=" + logCount + ", maxlogs=" +
666 this.maxLogs + "; forcing flush of " + regions.length + " regions(s): " +
667 sb.toString());
668 }
669 }
670 return regions;
671 }
672
673
674
675
676
677
678
679 Path cleanupCurrentWriter(final long currentfilenum) throws IOException {
680 Path oldFile = null;
681 if (this.writer != null) {
682
683 try {
684
685
686 if (this.unflushedEntries.get() != this.syncedTillHere) {
687 LOG.debug("cleanupCurrentWriter " +
688 " waiting for transactions to get synced " +
689 " total " + this.unflushedEntries.get() +
690 " synced till here " + syncedTillHere);
691 sync();
692 }
693 this.writer.close();
694 this.writer = null;
695 closeErrorCount.set(0);
696 } catch (IOException e) {
697 LOG.error("Failed close of HLog writer", e);
698 int errors = closeErrorCount.incrementAndGet();
699 if (errors <= closeErrorsTolerated && !hasDeferredEntries()) {
700 LOG.warn("Riding over HLog close failure! error count="+errors);
701 } else {
702 if (hasDeferredEntries()) {
703 LOG.error("Aborting due to unflushed edits in HLog");
704 }
705
706
707
708 FailedLogCloseException flce =
709 new FailedLogCloseException("#" + currentfilenum);
710 flce.initCause(e);
711 throw flce;
712 }
713 }
714 if (currentfilenum >= 0) {
715 oldFile = computeFilename(currentfilenum);
716 this.outputfiles.put(Long.valueOf(this.logSeqNum.get()), oldFile);
717 }
718 }
719 return oldFile;
720 }
721
722 private void archiveLogFile(final Path p, final Long seqno) throws IOException {
723 Path newPath = getHLogArchivePath(this.oldLogDir, p);
724 LOG.info("moving old hlog file " + FSUtils.getPath(p) +
725 " whose highest sequenceid is " + seqno + " to " +
726 FSUtils.getPath(newPath));
727
728
729 if (!this.listeners.isEmpty()) {
730 for (WALActionsListener i : this.listeners) {
731 i.preLogArchive(p, newPath);
732 }
733 }
734 if (!FSUtils.renameAndSetModifyTime(this.fs, p, newPath)) {
735 throw new IOException("Unable to rename " + p + " to " + newPath);
736 }
737
738 if (!this.listeners.isEmpty()) {
739 for (WALActionsListener i : this.listeners) {
740 i.postLogArchive(p, newPath);
741 }
742 }
743 }
744
745
746
747
748
749
750 protected Path computeFilename() {
751 return computeFilename(this.filenum);
752 }
753
754
755
756
757
758
759
760 protected Path computeFilename(long filenum) {
761 if (filenum < 0) {
762 throw new RuntimeException("hlog file number can't be < 0");
763 }
764 String child = prefix + "." + filenum;
765 if (forMeta) {
766 child += HLog.META_HLOG_FILE_EXTN;
767 }
768 return new Path(dir, child);
769 }
770
771 @Override
772 public void closeAndDelete() throws IOException {
773 close();
774 if (!fs.exists(this.dir)) return;
775 FileStatus[] files = fs.listStatus(this.dir);
776 if (files != null) {
777 for(FileStatus file : files) {
778
779 Path p = getHLogArchivePath(this.oldLogDir, file.getPath());
780
781 if (!this.listeners.isEmpty()) {
782 for (WALActionsListener i : this.listeners) {
783 i.preLogArchive(file.getPath(), p);
784 }
785 }
786
787 if (!FSUtils.renameAndSetModifyTime(fs, file.getPath(), p)) {
788 throw new IOException("Unable to rename " + file.getPath() + " to " + p);
789 }
790
791 if (!this.listeners.isEmpty()) {
792 for (WALActionsListener i : this.listeners) {
793 i.postLogArchive(file.getPath(), p);
794 }
795 }
796 }
797 LOG.debug("Moved " + files.length + " WAL file(s) to " + FSUtils.getPath(this.oldLogDir));
798 }
799 if (!fs.delete(dir, true)) {
800 LOG.info("Unable to delete " + dir);
801 }
802 }
803
804 @Override
805 public void close() throws IOException {
806 if (this.closed) {
807 return;
808 }
809
810 if (this.optionalFlushInterval > 0) {
811 try {
812 logSyncer.close();
813
814 logSyncer.join(this.optionalFlushInterval * 2);
815 } catch (InterruptedException e) {
816 LOG.error("Exception while waiting for syncer thread to die", e);
817 Thread.currentThread().interrupt();
818 }
819 }
820 try {
821
822 closeBarrier.stopAndDrainOps();
823 } catch (InterruptedException e) {
824 LOG.error("Exception while waiting for cache flushes and log rolls", e);
825 Thread.currentThread().interrupt();
826 }
827
828
829 if (!this.listeners.isEmpty()) {
830 for (WALActionsListener i : this.listeners) {
831 i.logCloseRequested();
832 }
833 }
834 synchronized (updateLock) {
835 this.closed = true;
836 if (LOG.isDebugEnabled()) {
837 LOG.debug("Closing WAL writer in " + this.dir.toString());
838 }
839 if (this.writer != null) {
840 this.writer.close();
841 this.writer = null;
842 }
843 }
844 }
845
846
847
848
849
850
851
852
853
854 protected HLogKey makeKey(byte[] encodedRegionName, TableName tableName, long seqnum,
855 long now, List<UUID> clusterIds) {
856 return new HLogKey(encodedRegionName, tableName, seqnum, now, clusterIds);
857 }
858
859 @Override
860 public void append(HRegionInfo info, TableName tableName, WALEdit edits,
861 final long now, HTableDescriptor htd)
862 throws IOException {
863 append(info, tableName, edits, now, htd, true);
864 }
865
866 @Override
867 public void append(HRegionInfo info, TableName tableName, WALEdit edits,
868 final long now, HTableDescriptor htd, boolean isInMemstore) throws IOException {
869 append(info, tableName, edits, new ArrayList<UUID>(), now, htd, true, isInMemstore);
870 }
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898 @SuppressWarnings("deprecation")
899 private long append(HRegionInfo info, TableName tableName, WALEdit edits, List<UUID> clusterIds,
900 final long now, HTableDescriptor htd, boolean doSync, boolean isInMemstore)
901 throws IOException {
902 if (edits.isEmpty()) return this.unflushedEntries.get();
903 if (this.closed) {
904 throw new IOException("Cannot append; log is closed");
905 }
906 TraceScope traceScope = Trace.startSpan("FSHlog.append");
907 try {
908 long txid = 0;
909 synchronized (this.updateLock) {
910 long seqNum = obtainSeqNum();
911
912
913
914
915
916
917
918 byte [] encodedRegionName = info.getEncodedNameAsBytes();
919 if (isInMemstore) this.oldestUnflushedSeqNums.putIfAbsent(encodedRegionName, seqNum);
920 HLogKey logKey = makeKey(encodedRegionName, tableName, seqNum, now, clusterIds);
921 doWrite(info, logKey, edits, htd);
922 this.numEntries.incrementAndGet();
923 txid = this.unflushedEntries.incrementAndGet();
924 if (htd.isDeferredLogFlush()) {
925 lastDeferredTxid = txid;
926 }
927 }
928
929
930 if (doSync &&
931 (info.isMetaRegion() ||
932 !htd.isDeferredLogFlush())) {
933
934 this.sync(txid);
935 }
936 return txid;
937 } finally {
938 traceScope.close();
939 }
940 }
941
942 @Override
943 public long appendNoSync(HRegionInfo info, TableName tableName, WALEdit edits,
944 List<UUID> clusterIds, final long now, HTableDescriptor htd)
945 throws IOException {
946 return append(info, tableName, edits, clusterIds, now, htd, false, true);
947 }
948
949
950
951
952
953
954
955
956
957 class LogSyncer extends HasThread {
958
959 private final long optionalFlushInterval;
960
961 private final AtomicBoolean closeLogSyncer = new AtomicBoolean(false);
962
963
964
965
966
967
968
969 private List<Entry> pendingWrites = new LinkedList<Entry>();
970
971 LogSyncer(long optionalFlushInterval) {
972 this.optionalFlushInterval = optionalFlushInterval;
973 }
974
975 @Override
976 public void run() {
977 try {
978
979
980 while(!this.isInterrupted() && !closeLogSyncer.get()) {
981
982 try {
983 if (unflushedEntries.get() <= syncedTillHere) {
984 synchronized (closeLogSyncer) {
985 closeLogSyncer.wait(this.optionalFlushInterval);
986 }
987 }
988
989
990
991 sync();
992 } catch (IOException e) {
993 LOG.error("Error while syncing, requesting close of hlog ", e);
994 requestLogRoll();
995 Threads.sleep(this.optionalFlushInterval);
996 }
997 }
998 } catch (InterruptedException e) {
999 LOG.debug(getName() + " interrupted while waiting for sync requests");
1000 } finally {
1001 LOG.info(getName() + " exiting");
1002 }
1003 }
1004
1005
1006
1007
1008 synchronized void append(Entry e) throws IOException {
1009 pendingWrites.add(e);
1010 }
1011
1012
1013
1014 synchronized List<Entry> getPendingWrites() {
1015 List<Entry> save = this.pendingWrites;
1016 this.pendingWrites = new LinkedList<Entry>();
1017 return save;
1018 }
1019
1020
1021 void hlogFlush(Writer writer, List<Entry> pending) throws IOException {
1022 if (pending == null) return;
1023
1024
1025 for (Entry e : pending) {
1026 writer.append(e);
1027 }
1028 }
1029
1030 void close() {
1031 synchronized (closeLogSyncer) {
1032 closeLogSyncer.set(true);
1033 closeLogSyncer.notifyAll();
1034 }
1035 }
1036 }
1037
1038
1039 private void syncer() throws IOException {
1040 syncer(this.unflushedEntries.get());
1041 }
1042
1043
1044 private void syncer(long txid) throws IOException {
1045
1046
1047 if (txid <= this.syncedTillHere) {
1048 return;
1049 }
1050 Writer tempWriter;
1051 synchronized (this.updateLock) {
1052 if (this.closed) return;
1053
1054
1055
1056
1057 tempWriter = this.writer;
1058 }
1059 try {
1060 long doneUpto;
1061 long now = EnvironmentEdgeManager.currentTimeMillis();
1062
1063
1064
1065
1066 IOException ioe = null;
1067 List<Entry> pending = null;
1068 synchronized (flushLock) {
1069 if (txid <= this.syncedTillHere) {
1070 return;
1071 }
1072 doneUpto = this.unflushedEntries.get();
1073 pending = logSyncer.getPendingWrites();
1074 try {
1075 logSyncer.hlogFlush(tempWriter, pending);
1076 postAppend(pending);
1077 } catch(IOException io) {
1078 ioe = io;
1079 LOG.error("syncer encountered error, will retry. txid=" + txid, ioe);
1080 }
1081 }
1082 if (ioe != null && pending != null) {
1083 synchronized (this.updateLock) {
1084 synchronized (flushLock) {
1085
1086 tempWriter = this.writer;
1087 logSyncer.hlogFlush(tempWriter, pending);
1088 postAppend(pending);
1089 }
1090 }
1091 }
1092
1093 if (txid <= this.syncedTillHere) {
1094 return;
1095 }
1096 try {
1097 if (tempWriter != null) {
1098 tempWriter.sync();
1099 postSync();
1100 }
1101 } catch(IOException ex) {
1102 synchronized (this.updateLock) {
1103
1104
1105
1106 tempWriter = this.writer;
1107 if (tempWriter != null) {
1108 tempWriter.sync();
1109 postSync();
1110 }
1111 }
1112 }
1113 this.syncedTillHere = Math.max(this.syncedTillHere, doneUpto);
1114
1115 this.metrics.finishSync(EnvironmentEdgeManager.currentTimeMillis() - now);
1116 boolean logRollNeeded = false;
1117 if (rollWriterLock.tryLock()) {
1118 try {
1119 logRollNeeded = checkLowReplication();
1120 } finally {
1121 rollWriterLock.unlock();
1122 }
1123 try {
1124 curLogSize = tempWriter.getLength();
1125 if (logRollNeeded || curLogSize > this.logrollsize) {
1126 requestLogRoll();
1127 }
1128 } catch (IOException x) {
1129 LOG.debug("Log roll failed and will be retried. (This is not an error)");
1130 }
1131 }
1132 } catch (IOException e) {
1133 LOG.fatal("Could not sync. Requesting roll of hlog", e);
1134 requestLogRoll();
1135 throw e;
1136 }
1137 }
1138
1139 @Override
1140 public void postSync() {}
1141
1142 @Override
1143 public void postAppend(List<Entry> entries) {}
1144
1145
1146
1147
1148 private boolean checkLowReplication() {
1149 boolean logRollNeeded = false;
1150
1151
1152 try {
1153 int numCurrentReplicas = getLogReplication();
1154 if (numCurrentReplicas != 0
1155 && numCurrentReplicas < this.minTolerableReplication) {
1156 if (this.lowReplicationRollEnabled) {
1157 if (this.consecutiveLogRolls.get() < this.lowReplicationRollLimit) {
1158 LOG.warn("HDFS pipeline error detected. " + "Found "
1159 + numCurrentReplicas + " replicas but expecting no less than "
1160 + this.minTolerableReplication + " replicas. "
1161 + " Requesting close of hlog.");
1162 logRollNeeded = true;
1163
1164
1165
1166 this.consecutiveLogRolls.getAndIncrement();
1167 } else {
1168 LOG.warn("Too many consecutive RollWriter requests, it's a sign of "
1169 + "the total number of live datanodes is lower than the tolerable replicas.");
1170 this.consecutiveLogRolls.set(0);
1171 this.lowReplicationRollEnabled = false;
1172 }
1173 }
1174 } else if (numCurrentReplicas >= this.minTolerableReplication) {
1175
1176 if (!this.lowReplicationRollEnabled) {
1177
1178
1179
1180 if (this.numEntries.get() <= 1) {
1181 return logRollNeeded;
1182 }
1183
1184
1185 this.lowReplicationRollEnabled = true;
1186 LOG.info("LowReplication-Roller was enabled.");
1187 }
1188 }
1189 } catch (Exception e) {
1190 LOG.warn("Unable to invoke DFSOutputStream.getNumCurrentReplicas" + e +
1191 " still proceeding ahead...");
1192 }
1193 return logRollNeeded;
1194 }
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208 int getLogReplication()
1209 throws IllegalArgumentException, IllegalAccessException, InvocationTargetException {
1210 if (this.getNumCurrentReplicas != null && this.hdfs_out != null) {
1211 Object repl = this.getNumCurrentReplicas.invoke(getOutputStream(), NO_ARGS);
1212 if (repl instanceof Integer) {
1213 return ((Integer)repl).intValue();
1214 }
1215 }
1216 return 0;
1217 }
1218
1219 boolean canGetCurReplicas() {
1220 return this.getNumCurrentReplicas != null;
1221 }
1222
1223 @Override
1224 public void hsync() throws IOException {
1225 syncer();
1226 }
1227
1228 @Override
1229 public void hflush() throws IOException {
1230 syncer();
1231 }
1232
1233 @Override
1234 public void sync() throws IOException {
1235 syncer();
1236 }
1237
1238 @Override
1239 public void sync(long txid) throws IOException {
1240 syncer(txid);
1241 }
1242
1243 private void requestLogRoll() {
1244 if (!this.listeners.isEmpty()) {
1245 for (WALActionsListener i: this.listeners) {
1246 i.logRollRequested();
1247 }
1248 }
1249 }
1250
1251
1252 protected void doWrite(HRegionInfo info, HLogKey logKey, WALEdit logEdit,
1253 HTableDescriptor htd)
1254 throws IOException {
1255 if (!this.enabled) {
1256 return;
1257 }
1258 if (!this.listeners.isEmpty()) {
1259 for (WALActionsListener i: this.listeners) {
1260 i.visitLogEntryBeforeWrite(htd, logKey, logEdit);
1261 }
1262 }
1263 try {
1264 long now = EnvironmentEdgeManager.currentTimeMillis();
1265
1266 if (!coprocessorHost.preWALWrite(info, logKey, logEdit)) {
1267 if (logEdit.isReplay()) {
1268
1269 logKey.setScopes(null);
1270 }
1271
1272 logSyncer.append(new FSHLog.Entry(logKey, logEdit));
1273 }
1274 long took = EnvironmentEdgeManager.currentTimeMillis() - now;
1275 coprocessorHost.postWALWrite(info, logKey, logEdit);
1276 long len = 0;
1277 for (KeyValue kv : logEdit.getKeyValues()) {
1278 len += kv.getLength();
1279 }
1280 this.metrics.finishAppend(took, len);
1281 } catch (IOException e) {
1282 LOG.fatal("Could not append. Requesting close of hlog", e);
1283 requestLogRoll();
1284 throw e;
1285 }
1286 }
1287
1288
1289
1290 int getNumEntries() {
1291 return numEntries.get();
1292 }
1293
1294 @Override
1295 public long obtainSeqNum() {
1296 return this.logSeqNum.incrementAndGet();
1297 }
1298
1299
1300 public int getNumRolledLogFiles() {
1301 return outputfiles.size();
1302 }
1303
1304
1305 @Override
1306 public int getNumLogFiles() {
1307
1308 return getNumRolledLogFiles() + 1;
1309 }
1310
1311
1312 @Override
1313 public long getLogFileSize() {
1314 return totalLogSize.get() + curLogSize;
1315 }
1316
1317 @Override
1318 public Long startCacheFlush(final byte[] encodedRegionName) {
1319 Long oldRegionSeqNum = null;
1320 if (!closeBarrier.beginOp()) {
1321 return null;
1322 }
1323 synchronized (oldestSeqNumsLock) {
1324 oldRegionSeqNum = this.oldestUnflushedSeqNums.remove(encodedRegionName);
1325 if (oldRegionSeqNum != null) {
1326 Long oldValue = this.oldestFlushingSeqNums.put(encodedRegionName, oldRegionSeqNum);
1327 assert oldValue == null : "Flushing map not cleaned up for "
1328 + Bytes.toString(encodedRegionName);
1329 }
1330 }
1331 if (oldRegionSeqNum == null) {
1332
1333
1334
1335
1336
1337 LOG.warn("Couldn't find oldest seqNum for the region we are about to flush: ["
1338 + Bytes.toString(encodedRegionName) + "]");
1339 }
1340 return obtainSeqNum();
1341 }
1342
1343 @Override
1344 public void completeCacheFlush(final byte [] encodedRegionName)
1345 {
1346 synchronized (oldestSeqNumsLock) {
1347 this.oldestFlushingSeqNums.remove(encodedRegionName);
1348 }
1349 closeBarrier.endOp();
1350 }
1351
1352 @Override
1353 public void abortCacheFlush(byte[] encodedRegionName) {
1354 Long currentSeqNum = null, seqNumBeforeFlushStarts = null;
1355 synchronized (oldestSeqNumsLock) {
1356 seqNumBeforeFlushStarts = this.oldestFlushingSeqNums.remove(encodedRegionName);
1357 if (seqNumBeforeFlushStarts != null) {
1358 currentSeqNum =
1359 this.oldestUnflushedSeqNums.put(encodedRegionName, seqNumBeforeFlushStarts);
1360 }
1361 }
1362 closeBarrier.endOp();
1363 if ((currentSeqNum != null)
1364 && (currentSeqNum.longValue() <= seqNumBeforeFlushStarts.longValue())) {
1365 String errorStr = "Region " + Bytes.toString(encodedRegionName) +
1366 "acquired edits out of order current memstore seq=" + currentSeqNum
1367 + ", previous oldest unflushed id=" + seqNumBeforeFlushStarts;
1368 LOG.error(errorStr);
1369 assert false : errorStr;
1370 Runtime.getRuntime().halt(1);
1371 }
1372 }
1373
1374 @Override
1375 public boolean isLowReplicationRollEnabled() {
1376 return lowReplicationRollEnabled;
1377 }
1378
1379
1380
1381
1382
1383
1384 protected Path getDir() {
1385 return dir;
1386 }
1387
1388 static Path getHLogArchivePath(Path oldLogDir, Path p) {
1389 return new Path(oldLogDir, p.getName());
1390 }
1391
1392 static String formatRecoveredEditsFileName(final long seqid) {
1393 return String.format("%019d", seqid);
1394 }
1395
1396 public static final long FIXED_OVERHEAD = ClassSize.align(
1397 ClassSize.OBJECT + (5 * ClassSize.REFERENCE) +
1398 ClassSize.ATOMIC_INTEGER + Bytes.SIZEOF_INT + (3 * Bytes.SIZEOF_LONG));
1399
1400 private static void usage() {
1401 System.err.println("Usage: HLog <ARGS>");
1402 System.err.println("Arguments:");
1403 System.err.println(" --dump Dump textual representation of passed one or more files");
1404 System.err.println(" For example: HLog --dump hdfs://example.com:9000/hbase/.logs/MACHINE/LOGFILE");
1405 System.err.println(" --split Split the passed directory of WAL logs");
1406 System.err.println(" For example: HLog --split hdfs://example.com:9000/hbase/.logs/DIR");
1407 }
1408
1409 private static void split(final Configuration conf, final Path p)
1410 throws IOException {
1411 FileSystem fs = FileSystem.get(conf);
1412 if (!fs.exists(p)) {
1413 throw new FileNotFoundException(p.toString());
1414 }
1415 if (!fs.getFileStatus(p).isDir()) {
1416 throw new IOException(p + " is not a directory");
1417 }
1418
1419 final Path baseDir = FSUtils.getRootDir(conf);
1420 final Path oldLogDir = new Path(baseDir, HConstants.HREGION_OLDLOGDIR_NAME);
1421 HLogSplitter.split(baseDir, p, oldLogDir, fs, conf);
1422 }
1423
1424 @Override
1425 public WALCoprocessorHost getCoprocessorHost() {
1426 return coprocessorHost;
1427 }
1428
1429
1430 boolean hasDeferredEntries() {
1431 return lastDeferredTxid > syncedTillHere;
1432 }
1433
1434 @Override
1435 public long getEarliestMemstoreSeqNum(byte[] encodedRegionName) {
1436 Long result = oldestUnflushedSeqNums.get(encodedRegionName);
1437 return result == null ? HConstants.NO_SEQNUM : result.longValue();
1438 }
1439
1440
1441
1442
1443
1444
1445
1446
1447 public static void main(String[] args) throws IOException {
1448 if (args.length < 2) {
1449 usage();
1450 System.exit(-1);
1451 }
1452
1453 if (args[0].compareTo("--dump") == 0) {
1454 HLogPrettyPrinter.run(Arrays.copyOfRange(args, 1, args.length));
1455 } else if (args[0].compareTo("--split") == 0) {
1456 Configuration conf = HBaseConfiguration.create();
1457 for (int i = 1; i < args.length; i++) {
1458 try {
1459 Path logPath = new Path(args[i]);
1460 FSUtils.setFsDefault(conf, logPath);
1461 split(conf, logPath);
1462 } catch (Throwable t) {
1463 t.printStackTrace(System.err);
1464 System.exit(-1);
1465 }
1466 }
1467 } else {
1468 usage();
1469 System.exit(-1);
1470 }
1471 }
1472 }