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.TreeMap;
31 import java.util.LinkedList;
32 import java.util.List;
33 import java.util.Map;
34 import java.util.SortedMap;
35 import java.util.TreeMap;
36 import java.util.TreeSet;
37 import java.util.UUID;
38 import java.util.concurrent.ConcurrentSkipListMap;
39 import java.util.concurrent.CopyOnWriteArrayList;
40 import java.util.concurrent.atomic.AtomicBoolean;
41 import java.util.concurrent.atomic.AtomicInteger;
42 import java.util.concurrent.atomic.AtomicLong;
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.exceptions.FailedLogCloseException;
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
68
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 @InterfaceAudience.Private
108 class FSHLog implements HLog, Syncable {
109 static final Log LOG = LogFactory.getLog(FSHLog.class);
110
111 private final FileSystem fs;
112 private final Path rootDir;
113 private final Path dir;
114 private final Configuration conf;
115
116 private List<WALActionsListener> listeners =
117 new CopyOnWriteArrayList<WALActionsListener>();
118 private final long optionalFlushInterval;
119 private final long blocksize;
120 private final String prefix;
121 private final AtomicLong unflushedEntries = new AtomicLong(0);
122 private volatile long syncedTillHere = 0;
123 private long lastDeferredTxid;
124 private final Path oldLogDir;
125 private volatile boolean logRollRunning;
126 private boolean failIfLogDirExists;
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 Object rollWriterLock = new Object();
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
208
209 private final Object updateLock = new Object();
210 private final Object flushLock = new Object();
211
212 private final boolean enabled;
213
214
215
216
217
218
219 private final int maxLogs;
220
221
222
223
224 private final LogSyncer logSyncer;
225
226
227 private final int closeErrorsTolerated;
228
229 private final AtomicInteger closeErrorCount = new AtomicInteger();
230 private final MetricsWAL metrics;
231
232
233
234
235
236
237
238
239
240
241 public FSHLog(final FileSystem fs, final Path root, final String logDir,
242 final Configuration conf)
243 throws IOException {
244 this(fs, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME,
245 conf, null, true, null, false);
246 }
247
248
249
250
251
252
253
254
255
256
257
258 public FSHLog(final FileSystem fs, final Path root, final String logDir,
259 final String oldLogDir, final Configuration conf)
260 throws IOException {
261 this(fs, root, logDir, oldLogDir,
262 conf, null, true, null, false);
263 }
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284 public FSHLog(final FileSystem fs, final Path root, final String logDir,
285 final Configuration conf, final List<WALActionsListener> listeners,
286 final String prefix) throws IOException {
287 this(fs, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME,
288 conf, listeners, true, prefix, false);
289 }
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313 public FSHLog(final FileSystem fs, final Path root, final String logDir,
314 final String oldLogDir, final Configuration conf,
315 final List<WALActionsListener> listeners,
316 final boolean failIfLogDirExists, final String prefix, boolean forMeta)
317 throws IOException {
318 super();
319 this.fs = fs;
320 this.rootDir = root;
321 this.dir = new Path(this.rootDir, logDir);
322 this.oldLogDir = new Path(this.rootDir, oldLogDir);
323 this.forMeta = forMeta;
324 this.conf = conf;
325
326 if (listeners != null) {
327 for (WALActionsListener i: listeners) {
328 registerWALActionsListener(i);
329 }
330 }
331
332 this.failIfLogDirExists = failIfLogDirExists;
333
334 this.blocksize = this.conf.getLong("hbase.regionserver.hlog.blocksize",
335 getDefaultBlockSize());
336
337 float multi = conf.getFloat("hbase.regionserver.logroll.multiplier", 0.95f);
338 this.logrollsize = (long)(this.blocksize * multi);
339 this.optionalFlushInterval =
340 conf.getLong("hbase.regionserver.optionallogflushinterval", 1 * 1000);
341
342 this.maxLogs = conf.getInt("hbase.regionserver.maxlogs", 32);
343 this.minTolerableReplication = conf.getInt(
344 "hbase.regionserver.hlog.tolerable.lowreplication",
345 this.fs.getDefaultReplication());
346 this.lowReplicationRollLimit = conf.getInt(
347 "hbase.regionserver.hlog.lowreplication.rolllimit", 5);
348 this.enabled = conf.getBoolean("hbase.regionserver.hlog.enabled", true);
349 this.closeErrorsTolerated = conf.getInt(
350 "hbase.regionserver.logroll.errors.tolerated", 0);
351
352 this.logSyncer = new LogSyncer(this.optionalFlushInterval);
353
354 LOG.info("HLog configuration: blocksize=" +
355 StringUtils.byteDesc(this.blocksize) +
356 ", rollsize=" + StringUtils.byteDesc(this.logrollsize) +
357 ", enabled=" + this.enabled +
358 ", optionallogflushinternal=" + this.optionalFlushInterval + "ms");
359
360 this.prefix = prefix == null || prefix.isEmpty() ?
361 "hlog" : URLEncoder.encode(prefix, "UTF8");
362
363 boolean dirExists = false;
364 if (failIfLogDirExists && (dirExists = this.fs.exists(dir))) {
365 throw new IOException("Target HLog directory already exists: " + dir);
366 }
367 if (!dirExists && !fs.mkdirs(dir)) {
368 throw new IOException("Unable to mkdir " + dir);
369 }
370
371 if (!fs.exists(this.oldLogDir)) {
372 if (!fs.mkdirs(this.oldLogDir)) {
373 throw new IOException("Unable to mkdir " + this.oldLogDir);
374 }
375 }
376
377 rollWriter();
378
379
380 this.getNumCurrentReplicas = getGetNumCurrentReplicas(this.hdfs_out);
381
382
383 if (this.optionalFlushInterval > 0) {
384 Threads.setDaemonThreadRunning(logSyncer.getThread(), Thread.currentThread().getName()
385 + ".logSyncer");
386 } else {
387 LOG.info("hbase.regionserver.optionallogflushinterval is set as "
388 + this.optionalFlushInterval + ". Deferred log syncing won't work. "
389 + "Any Mutation, marked to be deferred synced, will be flushed immediately.");
390 }
391 coprocessorHost = new WALCoprocessorHost(this, conf);
392
393 this.metrics = new MetricsWAL();
394 }
395
396
397
398 private long getDefaultBlockSize() throws IOException {
399 Method m = null;
400 Class<? extends FileSystem> cls = this.fs.getClass();
401 try {
402 m = cls.getMethod("getDefaultBlockSize",
403 new Class<?>[] { Path.class });
404 } catch (NoSuchMethodException e) {
405 LOG.info("FileSystem doesn't support getDefaultBlockSize");
406 } catch (SecurityException e) {
407 LOG.info("Doesn't have access to getDefaultBlockSize on "
408 + "FileSystems", e);
409 m = null;
410 }
411 if (null == m) {
412 return this.fs.getDefaultBlockSize();
413 } else {
414 try {
415 Object ret = m.invoke(this.fs, this.dir);
416 return ((Long)ret).longValue();
417 } catch (Exception e) {
418 throw new IOException(e);
419 }
420 }
421 }
422
423
424
425
426
427 private Method getGetNumCurrentReplicas(final FSDataOutputStream os) {
428 Method m = null;
429 if (os != null) {
430 Class<? extends OutputStream> wrappedStreamClass = os.getWrappedStream()
431 .getClass();
432 try {
433 m = wrappedStreamClass.getDeclaredMethod("getNumCurrentReplicas",
434 new Class<?>[] {});
435 m.setAccessible(true);
436 } catch (NoSuchMethodException e) {
437 LOG.info("FileSystem's output stream doesn't support"
438 + " getNumCurrentReplicas; --HDFS-826 not available; fsOut="
439 + wrappedStreamClass.getName());
440 } catch (SecurityException e) {
441 LOG.info("Doesn't have access to getNumCurrentReplicas on "
442 + "FileSystems's output stream --HDFS-826 not available; fsOut="
443 + wrappedStreamClass.getName(), e);
444 m = null;
445 }
446 }
447 if (m != null) {
448 LOG.info("Using getNumCurrentReplicas--HDFS-826");
449 }
450 return m;
451 }
452
453 @Override
454 public void registerWALActionsListener(final WALActionsListener listener) {
455 this.listeners.add(listener);
456 }
457
458 @Override
459 public boolean unregisterWALActionsListener(final WALActionsListener listener) {
460 return this.listeners.remove(listener);
461 }
462
463 @Override
464 public long getFilenum() {
465 return this.filenum;
466 }
467
468 @Override
469 public void setSequenceNumber(final long newvalue) {
470 for (long id = this.logSeqNum.get(); id < newvalue &&
471 !this.logSeqNum.compareAndSet(id, newvalue); id = this.logSeqNum.get()) {
472
473
474 LOG.debug("Changed sequenceid from " + id + " to " + newvalue);
475 }
476 }
477
478 @Override
479 public long getSequenceNumber() {
480 return logSeqNum.get();
481 }
482
483
484
485
486
487
488
489
490
491 OutputStream getOutputStream() {
492 return this.hdfs_out.getWrappedStream();
493 }
494
495 @Override
496 public byte [][] rollWriter() throws FailedLogCloseException, IOException {
497 return rollWriter(false);
498 }
499
500 @Override
501 public byte [][] rollWriter(boolean force)
502 throws FailedLogCloseException, IOException {
503 synchronized (rollWriterLock) {
504
505 if (!force && this.writer != null && this.numEntries.get() <= 0) {
506 return null;
507 }
508 byte [][] regionsToFlush = null;
509 try {
510 this.logRollRunning = true;
511 boolean isClosed = closed;
512 if (isClosed || !closeBarrier.beginOp()) {
513 LOG.debug("HLog " + (isClosed ? "closed" : "closing") + ". Skipping rolling of writer");
514 return regionsToFlush;
515 }
516
517
518 long currentFilenum = this.filenum;
519 Path oldPath = null;
520 if (currentFilenum > 0) {
521
522 oldPath = computeFilename(currentFilenum);
523 }
524 this.filenum = System.currentTimeMillis();
525 Path newPath = computeFilename();
526
527
528 if (!this.listeners.isEmpty()) {
529 for (WALActionsListener i : this.listeners) {
530 i.preLogRoll(oldPath, newPath);
531 }
532 }
533 FSHLog.Writer nextWriter = this.createWriterInstance(fs, newPath, conf);
534
535
536
537 FSDataOutputStream nextHdfsOut = null;
538 if (nextWriter instanceof SequenceFileLogWriter) {
539 nextHdfsOut = ((SequenceFileLogWriter)nextWriter).getWriterFSDataOutputStream();
540 }
541
542 Path oldFile = null;
543 int oldNumEntries = 0;
544 synchronized (updateLock) {
545
546 oldNumEntries = this.numEntries.get();
547 oldFile = cleanupCurrentWriter(currentFilenum);
548 this.writer = nextWriter;
549 this.hdfs_out = nextHdfsOut;
550 this.numEntries.set(0);
551 }
552 LOG.info("Rolled log" + (oldFile != null ? " for file=" + FSUtils.getPath(oldFile)
553 + ", entries=" + oldNumEntries + ", filesize=" + this.fs.getFileStatus(oldFile).getLen()
554 : "" ) + "; new path=" + FSUtils.getPath(newPath));
555
556
557 if (!this.listeners.isEmpty()) {
558 for (WALActionsListener i : this.listeners) {
559 i.postLogRoll(oldPath, newPath);
560 }
561 }
562
563
564 if (getNumLogFiles() > 0) {
565 cleanOldLogs();
566 regionsToFlush = getRegionsToForceFlush();
567 }
568 } finally {
569 this.logRollRunning = false;
570 closeBarrier.endOp();
571 }
572 return regionsToFlush;
573 }
574 }
575
576
577
578
579
580
581
582
583
584
585
586 protected Writer createWriterInstance(final FileSystem fs, final Path path,
587 final Configuration conf) throws IOException {
588 if (forMeta) {
589
590 }
591 return HLogFactory.createWriter(fs, path, conf);
592 }
593
594
595
596
597
598
599
600
601 private void cleanOldLogs() throws IOException {
602 long oldestOutstandingSeqNum = Long.MAX_VALUE;
603 synchronized (oldestSeqNumsLock) {
604 Long oldestFlushing = (oldestFlushingSeqNums.size() > 0)
605 ? Collections.min(oldestFlushingSeqNums.values()) : Long.MAX_VALUE;
606 Long oldestUnflushed = (oldestUnflushedSeqNums.size() > 0)
607 ? Collections.min(oldestUnflushedSeqNums.values()) : Long.MAX_VALUE;
608 oldestOutstandingSeqNum = Math.min(oldestFlushing, oldestUnflushed);
609 }
610
611
612
613 TreeSet<Long> sequenceNumbers = new TreeSet<Long>(this.outputfiles.headMap(
614 oldestOutstandingSeqNum).keySet());
615
616 if (LOG.isDebugEnabled()) {
617 if (sequenceNumbers.size() > 0) {
618 LOG.debug("Found " + sequenceNumbers.size() + " hlogs to remove" +
619 " out of total " + this.outputfiles.size() + ";" +
620 " oldest outstanding sequenceid is " + oldestOutstandingSeqNum);
621 }
622 }
623 for (Long seq : sequenceNumbers) {
624 archiveLogFile(this.outputfiles.remove(seq), 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 = getNumLogFiles();
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 (!this.fs.rename(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 for(FileStatus file : files) {
777
778 Path p = getHLogArchivePath(this.oldLogDir, file.getPath());
779
780 if (!this.listeners.isEmpty()) {
781 for (WALActionsListener i : this.listeners) {
782 i.preLogArchive(file.getPath(), p);
783 }
784 }
785
786 if (!fs.rename(file.getPath(),p)) {
787 throw new IOException("Unable to rename " + file.getPath() + " to " + p);
788 }
789
790 if (!this.listeners.isEmpty()) {
791 for (WALActionsListener i : this.listeners) {
792 i.postLogArchive(file.getPath(), p);
793 }
794 }
795 }
796 LOG.debug("Moved " + files.length + " log files to " +
797 FSUtils.getPath(this.oldLogDir));
798 if (!fs.delete(dir, true)) {
799 LOG.info("Unable to delete " + dir);
800 }
801 }
802
803 @Override
804 public void close() throws IOException {
805 if (this.closed) {
806 return;
807 }
808
809 if (this.optionalFlushInterval > 0) {
810 try {
811 logSyncer.close();
812
813 logSyncer.join(this.optionalFlushInterval * 2);
814 } catch (InterruptedException e) {
815 LOG.error("Exception while waiting for syncer thread to die", e);
816 Thread.currentThread().interrupt();
817 }
818 }
819 try {
820
821 closeBarrier.stopAndDrainOps();
822 } catch (InterruptedException e) {
823 LOG.error("Exception while waiting for cache flushes and log rolls", e);
824 Thread.currentThread().interrupt();
825 }
826
827
828 if (!this.listeners.isEmpty()) {
829 for (WALActionsListener i : this.listeners) {
830 i.logCloseRequested();
831 }
832 }
833 synchronized (updateLock) {
834 this.closed = true;
835 if (LOG.isDebugEnabled()) {
836 LOG.debug("closing hlog writer in " + this.dir.toString());
837 }
838 if (this.writer != null) {
839 this.writer.close();
840 this.writer = null;
841 }
842 }
843 }
844
845
846
847
848
849
850
851
852 protected HLogKey makeKey(byte[] regionName, byte[] tableName, long seqnum,
853 long now, UUID clusterId) {
854 return new HLogKey(regionName, tableName, seqnum, now, clusterId);
855 }
856
857 @Override
858 public long append(HRegionInfo regionInfo, HLogKey logKey, WALEdit logEdit,
859 HTableDescriptor htd, boolean doSync)
860 throws IOException {
861 if (this.closed) {
862 throw new IOException("Cannot append; log is closed");
863 }
864 long txid = 0;
865 synchronized (updateLock) {
866 long seqNum = obtainSeqNum();
867 logKey.setLogSeqNum(seqNum);
868
869
870
871
872
873 this.oldestUnflushedSeqNums.putIfAbsent(regionInfo.getEncodedNameAsBytes(),
874 Long.valueOf(seqNum));
875 doWrite(regionInfo, logKey, logEdit, htd);
876 txid = this.unflushedEntries.incrementAndGet();
877 this.numEntries.incrementAndGet();
878 if (htd.isDeferredLogFlush()) {
879 lastDeferredTxid = txid;
880 }
881 }
882
883
884
885 if (doSync &&
886 (regionInfo.isMetaRegion() ||
887 !htd.isDeferredLogFlush())) {
888
889 this.sync(txid);
890 }
891 return txid;
892 }
893
894 @Override
895 public void append(HRegionInfo info, byte [] tableName, WALEdit edits,
896 final long now, HTableDescriptor htd)
897 throws IOException {
898 append(info, tableName, edits, HConstants.DEFAULT_CLUSTER_ID, now, htd);
899 }
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927 private long append(HRegionInfo info, byte [] tableName, WALEdit edits, UUID clusterId,
928 final long now, HTableDescriptor htd, boolean doSync)
929 throws IOException {
930 if (edits.isEmpty()) return this.unflushedEntries.get();;
931 if (this.closed) {
932 throw new IOException("Cannot append; log is closed");
933 }
934 long txid = 0;
935 synchronized (this.updateLock) {
936 long seqNum = obtainSeqNum();
937
938
939
940
941
942
943
944 byte [] encodedRegionName = info.getEncodedNameAsBytes();
945 this.oldestUnflushedSeqNums.putIfAbsent(encodedRegionName, seqNum);
946 HLogKey logKey = makeKey(encodedRegionName, tableName, seqNum, now, clusterId);
947 doWrite(info, logKey, edits, htd);
948 this.numEntries.incrementAndGet();
949 txid = this.unflushedEntries.incrementAndGet();
950 if (htd.isDeferredLogFlush()) {
951 lastDeferredTxid = txid;
952 }
953 }
954
955
956 if (doSync &&
957 (info.isMetaRegion() ||
958 !htd.isDeferredLogFlush())) {
959
960 this.sync(txid);
961 }
962 return txid;
963 }
964
965 @Override
966 public long appendNoSync(HRegionInfo info, byte [] tableName, WALEdit edits,
967 UUID clusterId, final long now, HTableDescriptor htd)
968 throws IOException {
969 return append(info, tableName, edits, clusterId, now, htd, false);
970 }
971
972 @Override
973 public long append(HRegionInfo info, byte [] tableName, WALEdit edits,
974 UUID clusterId, final long now, HTableDescriptor htd)
975 throws IOException {
976 return append(info, tableName, edits, clusterId, now, htd, true);
977 }
978
979
980
981
982
983
984
985
986
987 class LogSyncer extends HasThread {
988
989 private final long optionalFlushInterval;
990
991 private final AtomicBoolean closeLogSyncer = new AtomicBoolean(false);
992
993
994
995
996
997
998
999 private List<Entry> pendingWrites = new LinkedList<Entry>();
1000
1001 LogSyncer(long optionalFlushInterval) {
1002 this.optionalFlushInterval = optionalFlushInterval;
1003 }
1004
1005 @Override
1006 public void run() {
1007 try {
1008
1009
1010 while(!this.isInterrupted() && !closeLogSyncer.get()) {
1011
1012 try {
1013 if (unflushedEntries.get() <= syncedTillHere) {
1014 synchronized (closeLogSyncer) {
1015 closeLogSyncer.wait(this.optionalFlushInterval);
1016 }
1017 }
1018
1019
1020
1021 sync();
1022 } catch (IOException e) {
1023 LOG.error("Error while syncing, requesting close of hlog ", e);
1024 requestLogRoll();
1025 }
1026 }
1027 } catch (InterruptedException e) {
1028 LOG.debug(getName() + " interrupted while waiting for sync requests");
1029 } finally {
1030 LOG.info(getName() + " exiting");
1031 }
1032 }
1033
1034
1035
1036
1037 synchronized void append(Entry e) throws IOException {
1038 pendingWrites.add(e);
1039 }
1040
1041
1042
1043 synchronized List<Entry> getPendingWrites() {
1044 List<Entry> save = this.pendingWrites;
1045 this.pendingWrites = new LinkedList<Entry>();
1046 return save;
1047 }
1048
1049
1050 void hlogFlush(Writer writer, List<Entry> pending) throws IOException {
1051 if (pending == null) return;
1052
1053
1054 for (Entry e : pending) {
1055 writer.append(e);
1056 }
1057 }
1058
1059 void close() {
1060 synchronized (closeLogSyncer) {
1061 closeLogSyncer.set(true);
1062 closeLogSyncer.notifyAll();
1063 }
1064 }
1065 }
1066
1067
1068 private void syncer() throws IOException {
1069 syncer(this.unflushedEntries.get());
1070 }
1071
1072
1073 private void syncer(long txid) throws IOException {
1074 Writer tempWriter;
1075 synchronized (this.updateLock) {
1076 if (this.closed) return;
1077
1078
1079
1080
1081 tempWriter = this.writer;
1082 }
1083
1084
1085 if (txid <= this.syncedTillHere) {
1086 return;
1087 }
1088 try {
1089 long doneUpto;
1090 long now = EnvironmentEdgeManager.currentTimeMillis();
1091
1092
1093
1094
1095 IOException ioe = null;
1096 List<Entry> pending = null;
1097 synchronized (flushLock) {
1098 if (txid <= this.syncedTillHere) {
1099 return;
1100 }
1101 doneUpto = this.unflushedEntries.get();
1102 pending = logSyncer.getPendingWrites();
1103 try {
1104 logSyncer.hlogFlush(tempWriter, pending);
1105 } catch(IOException io) {
1106 ioe = io;
1107 LOG.error("syncer encountered error, will retry. txid=" + txid, ioe);
1108 }
1109 }
1110 if (ioe != null && pending != null) {
1111 synchronized (this.updateLock) {
1112 synchronized (flushLock) {
1113
1114 tempWriter = this.writer;
1115 logSyncer.hlogFlush(tempWriter, pending);
1116 }
1117 }
1118 }
1119
1120 if (txid <= this.syncedTillHere) {
1121 return;
1122 }
1123 try {
1124 if (tempWriter != null) tempWriter.sync();
1125 } catch(IOException ex) {
1126 synchronized (this.updateLock) {
1127
1128
1129
1130 tempWriter = this.writer;
1131 if (tempWriter != null) tempWriter.sync();
1132 }
1133 }
1134 this.syncedTillHere = Math.max(this.syncedTillHere, doneUpto);
1135
1136 this.metrics.finishSync(EnvironmentEdgeManager.currentTimeMillis() - now);
1137
1138
1139
1140 if (!this.logRollRunning) {
1141 checkLowReplication();
1142 try {
1143 if (tempWriter.getLength() > this.logrollsize) {
1144 requestLogRoll();
1145 }
1146 } catch (IOException x) {
1147 LOG.debug("Log roll failed and will be retried. (This is not an error)");
1148 }
1149 }
1150 } catch (IOException e) {
1151 LOG.fatal("Could not sync. Requesting close of hlog", e);
1152 requestLogRoll();
1153 throw e;
1154 }
1155 }
1156
1157 private void checkLowReplication() {
1158
1159
1160 try {
1161 int numCurrentReplicas = getLogReplication();
1162 if (numCurrentReplicas != 0
1163 && numCurrentReplicas < this.minTolerableReplication) {
1164 if (this.lowReplicationRollEnabled) {
1165 if (this.consecutiveLogRolls.get() < this.lowReplicationRollLimit) {
1166 LOG.warn("HDFS pipeline error detected. " + "Found "
1167 + numCurrentReplicas + " replicas but expecting no less than "
1168 + this.minTolerableReplication + " replicas. "
1169 + " Requesting close of hlog.");
1170 requestLogRoll();
1171
1172
1173
1174 this.consecutiveLogRolls.getAndIncrement();
1175 } else {
1176 LOG.warn("Too many consecutive RollWriter requests, it's a sign of "
1177 + "the total number of live datanodes is lower than the tolerable replicas.");
1178 this.consecutiveLogRolls.set(0);
1179 this.lowReplicationRollEnabled = false;
1180 }
1181 }
1182 } else if (numCurrentReplicas >= this.minTolerableReplication) {
1183
1184 if (!this.lowReplicationRollEnabled) {
1185
1186
1187
1188 if (this.numEntries.get() <= 1) {
1189 return;
1190 }
1191
1192
1193 this.lowReplicationRollEnabled = true;
1194 LOG.info("LowReplication-Roller was enabled.");
1195 }
1196 }
1197 } catch (Exception e) {
1198 LOG.warn("Unable to invoke DFSOutputStream.getNumCurrentReplicas" + e +
1199 " still proceeding ahead...");
1200 }
1201 }
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215 int getLogReplication()
1216 throws IllegalArgumentException, IllegalAccessException, InvocationTargetException {
1217 if (this.getNumCurrentReplicas != null && this.hdfs_out != null) {
1218 Object repl = this.getNumCurrentReplicas.invoke(getOutputStream(), NO_ARGS);
1219 if (repl instanceof Integer) {
1220 return ((Integer)repl).intValue();
1221 }
1222 }
1223 return 0;
1224 }
1225
1226 boolean canGetCurReplicas() {
1227 return this.getNumCurrentReplicas != null;
1228 }
1229
1230 public void hsync() throws IOException {
1231 syncer();
1232 }
1233
1234 public void hflush() throws IOException {
1235 syncer();
1236 }
1237
1238 public void sync() throws IOException {
1239 syncer();
1240 }
1241
1242 public void sync(long txid) throws IOException {
1243 syncer(txid);
1244 }
1245
1246 private void requestLogRoll() {
1247 if (!this.listeners.isEmpty()) {
1248 for (WALActionsListener i: this.listeners) {
1249 i.logRollRequested();
1250 }
1251 }
1252 }
1253
1254 protected void doWrite(HRegionInfo info, HLogKey logKey, WALEdit logEdit,
1255 HTableDescriptor htd)
1256 throws IOException {
1257 if (!this.enabled) {
1258 return;
1259 }
1260 if (!this.listeners.isEmpty()) {
1261 for (WALActionsListener i: this.listeners) {
1262 i.visitLogEntryBeforeWrite(htd, logKey, logEdit);
1263 }
1264 }
1265 try {
1266 long now = EnvironmentEdgeManager.currentTimeMillis();
1267
1268 if (!coprocessorHost.preWALWrite(info, logKey, logEdit)) {
1269
1270 logSyncer.append(new FSHLog.Entry(logKey, logEdit));
1271 }
1272 long took = EnvironmentEdgeManager.currentTimeMillis() - now;
1273 coprocessorHost.postWALWrite(info, logKey, logEdit);
1274 long len = 0;
1275 for (KeyValue kv : logEdit.getKeyValues()) {
1276 len += kv.getLength();
1277 }
1278 this.metrics.finishAppend(took, len);
1279 } catch (IOException e) {
1280 LOG.fatal("Could not append. Requesting close of hlog", e);
1281 requestLogRoll();
1282 throw e;
1283 }
1284 }
1285
1286
1287
1288 int getNumEntries() {
1289 return numEntries.get();
1290 }
1291
1292 @Override
1293 public long obtainSeqNum() {
1294 return this.logSeqNum.incrementAndGet();
1295 }
1296
1297
1298 int getNumLogFiles() {
1299 return outputfiles.size();
1300 }
1301
1302 @Override
1303 public Long startCacheFlush(final byte[] encodedRegionName) {
1304 Long oldRegionSeqNum = null;
1305 if (!closeBarrier.beginOp()) {
1306 return null;
1307 }
1308 synchronized (oldestSeqNumsLock) {
1309 oldRegionSeqNum = this.oldestUnflushedSeqNums.remove(encodedRegionName);
1310 if (oldRegionSeqNum != null) {
1311 Long oldValue = this.oldestFlushingSeqNums.put(encodedRegionName, oldRegionSeqNum);
1312 assert oldValue == null : "Flushing map not cleaned up for "
1313 + Bytes.toString(encodedRegionName);
1314 }
1315 }
1316 if (oldRegionSeqNum == null) {
1317
1318
1319
1320
1321
1322 LOG.warn("Couldn't find oldest seqNum for the region we are about to flush: ["
1323 + Bytes.toString(encodedRegionName) + "]");
1324 }
1325 return obtainSeqNum();
1326 }
1327
1328 @Override
1329 public void completeCacheFlush(final byte [] encodedRegionName)
1330 {
1331 synchronized (oldestSeqNumsLock) {
1332 this.oldestFlushingSeqNums.remove(encodedRegionName);
1333 }
1334 closeBarrier.endOp();
1335 }
1336
1337 @Override
1338 public void abortCacheFlush(byte[] encodedRegionName) {
1339 Long currentSeqNum = null, seqNumBeforeFlushStarts = null;
1340 synchronized (oldestSeqNumsLock) {
1341 seqNumBeforeFlushStarts = this.oldestFlushingSeqNums.remove(encodedRegionName);
1342 if (seqNumBeforeFlushStarts != null) {
1343 currentSeqNum =
1344 this.oldestUnflushedSeqNums.put(encodedRegionName, seqNumBeforeFlushStarts);
1345 }
1346 }
1347 closeBarrier.endOp();
1348 if ((currentSeqNum != null)
1349 && (currentSeqNum.longValue() <= seqNumBeforeFlushStarts.longValue())) {
1350 String errorStr = "Region " + Bytes.toString(encodedRegionName) +
1351 "acquired edits out of order current memstore seq=" + currentSeqNum
1352 + ", previous oldest unflushed id=" + seqNumBeforeFlushStarts;
1353 LOG.error(errorStr);
1354 assert false : errorStr;
1355 Runtime.getRuntime().halt(1);
1356 }
1357 }
1358
1359 @Override
1360 public boolean isLowReplicationRollEnabled() {
1361 return lowReplicationRollEnabled;
1362 }
1363
1364
1365
1366
1367
1368
1369 protected Path getDir() {
1370 return dir;
1371 }
1372
1373 static Path getHLogArchivePath(Path oldLogDir, Path p) {
1374 return new Path(oldLogDir, p.getName());
1375 }
1376
1377 static String formatRecoveredEditsFileName(final long seqid) {
1378 return String.format("%019d", seqid);
1379 }
1380
1381 public static final long FIXED_OVERHEAD = ClassSize.align(
1382 ClassSize.OBJECT + (5 * ClassSize.REFERENCE) +
1383 ClassSize.ATOMIC_INTEGER + Bytes.SIZEOF_INT + (3 * Bytes.SIZEOF_LONG));
1384
1385 private static void usage() {
1386 System.err.println("Usage: HLog <ARGS>");
1387 System.err.println("Arguments:");
1388 System.err.println(" --dump Dump textual representation of passed one or more files");
1389 System.err.println(" For example: HLog --dump hdfs://example.com:9000/hbase/.logs/MACHINE/LOGFILE");
1390 System.err.println(" --split Split the passed directory of WAL logs");
1391 System.err.println(" For example: HLog --split hdfs://example.com:9000/hbase/.logs/DIR");
1392 }
1393
1394 private static void split(final Configuration conf, final Path p)
1395 throws IOException {
1396 FileSystem fs = FileSystem.get(conf);
1397 if (!fs.exists(p)) {
1398 throw new FileNotFoundException(p.toString());
1399 }
1400 final Path baseDir = FSUtils.getRootDir(conf);
1401 final Path oldLogDir = new Path(baseDir, HConstants.HREGION_OLDLOGDIR_NAME);
1402 if (!fs.getFileStatus(p).isDir()) {
1403 throw new IOException(p + " is not a directory");
1404 }
1405
1406 HLogSplitter logSplitter = HLogSplitter.createLogSplitter(
1407 conf, baseDir, p, oldLogDir, fs);
1408 logSplitter.splitLog();
1409 }
1410
1411 @Override
1412 public WALCoprocessorHost getCoprocessorHost() {
1413 return coprocessorHost;
1414 }
1415
1416
1417 boolean hasDeferredEntries() {
1418 return lastDeferredTxid > syncedTillHere;
1419 }
1420
1421 @Override
1422 public long getEarliestMemstoreSeqNum(byte[] encodedRegionName) {
1423 Long result = oldestUnflushedSeqNums.get(encodedRegionName);
1424 return result == null ? HConstants.NO_SEQNUM : result.longValue();
1425 }
1426
1427
1428
1429
1430
1431
1432
1433
1434 public static void main(String[] args) throws IOException {
1435 if (args.length < 2) {
1436 usage();
1437 System.exit(-1);
1438 }
1439
1440 if (args[0].compareTo("--dump") == 0) {
1441 HLogPrettyPrinter.run(Arrays.copyOfRange(args, 1, args.length));
1442 } else if (args[0].compareTo("--split") == 0) {
1443 Configuration conf = HBaseConfiguration.create();
1444 for (int i = 1; i < args.length; i++) {
1445 try {
1446 Path logPath = new Path(args[i]);
1447 FSUtils.setFsDefault(conf, logPath);
1448 split(conf, logPath);
1449 } catch (Throwable t) {
1450 t.printStackTrace(System.err);
1451 System.exit(-1);
1452 }
1453 }
1454 } else {
1455 usage();
1456 System.exit(-1);
1457 }
1458 }
1459 }