1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.regionserver.wal;
19
20 import java.io.FileNotFoundException;
21 import java.io.IOException;
22 import java.io.InterruptedIOException;
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.Comparator;
30 import java.util.HashMap;
31 import java.util.List;
32 import java.util.Map;
33 import java.util.NavigableMap;
34 import java.util.TreeMap;
35 import java.util.UUID;
36 import java.util.concurrent.BlockingQueue;
37 import java.util.concurrent.ConcurrentHashMap;
38 import java.util.concurrent.ConcurrentSkipListMap;
39 import java.util.concurrent.CopyOnWriteArrayList;
40 import java.util.concurrent.CountDownLatch;
41 import java.util.concurrent.ExecutionException;
42 import java.util.concurrent.ExecutorService;
43 import java.util.concurrent.Executors;
44 import java.util.concurrent.LinkedBlockingQueue;
45 import java.util.concurrent.TimeUnit;
46 import java.util.concurrent.atomic.AtomicBoolean;
47 import java.util.concurrent.atomic.AtomicInteger;
48 import java.util.concurrent.atomic.AtomicLong;
49 import java.util.concurrent.locks.ReentrantLock;
50
51 import org.apache.commons.logging.Log;
52 import org.apache.commons.logging.LogFactory;
53 import org.apache.hadoop.hbase.classification.InterfaceAudience;
54 import org.apache.hadoop.conf.Configuration;
55 import org.apache.hadoop.fs.FSDataOutputStream;
56 import org.apache.hadoop.fs.FileStatus;
57 import org.apache.hadoop.fs.FileSystem;
58 import org.apache.hadoop.fs.Path;
59 import org.apache.hadoop.fs.PathFilter;
60 import org.apache.hadoop.hbase.Cell;
61 import org.apache.hadoop.hbase.CellUtil;
62 import org.apache.hadoop.hbase.HBaseConfiguration;
63 import org.apache.hadoop.hbase.HConstants;
64 import org.apache.hadoop.hbase.HRegionInfo;
65 import org.apache.hadoop.hbase.HTableDescriptor;
66 import org.apache.hadoop.hbase.KeyValue;
67 import org.apache.hadoop.hbase.TableName;
68 import static org.apache.hadoop.hbase.wal.DefaultWALProvider.WAL_FILE_NAME_DELIMITER;
69 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
70 import org.apache.hadoop.hbase.wal.WAL;
71 import org.apache.hadoop.hbase.wal.WAL.Entry;
72 import org.apache.hadoop.hbase.wal.WALFactory;
73 import org.apache.hadoop.hbase.wal.WALKey;
74 import org.apache.hadoop.hbase.wal.WALPrettyPrinter;
75 import org.apache.hadoop.hbase.wal.WALProvider.Writer;
76 import org.apache.hadoop.hbase.wal.WALSplitter;
77 import org.apache.hadoop.hbase.util.Bytes;
78 import org.apache.hadoop.hbase.util.ClassSize;
79 import org.apache.hadoop.hbase.util.DrainBarrier;
80 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
81 import org.apache.hadoop.hbase.util.FSUtils;
82 import org.apache.hadoop.hbase.util.HasThread;
83 import org.apache.hadoop.hbase.util.Threads;
84 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
85 import org.apache.hadoop.util.StringUtils;
86 import org.apache.htrace.NullScope;
87 import org.apache.htrace.Span;
88 import org.apache.htrace.Trace;
89 import org.apache.htrace.TraceScope;
90
91 import com.google.common.annotations.VisibleForTesting;
92 import com.lmax.disruptor.BlockingWaitStrategy;
93 import com.lmax.disruptor.EventHandler;
94 import com.lmax.disruptor.ExceptionHandler;
95 import com.lmax.disruptor.LifecycleAware;
96 import com.lmax.disruptor.TimeoutException;
97 import com.lmax.disruptor.dsl.Disruptor;
98 import com.lmax.disruptor.dsl.ProducerType;
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117 @InterfaceAudience.Private
118 public class FSHLog implements WAL {
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153 static final Log LOG = LogFactory.getLog(FSHLog.class);
154
155 private static final int DEFAULT_SLOW_SYNC_TIME_MS = 100;
156
157
158
159
160
161
162
163
164
165 private final Disruptor<RingBufferTruck> disruptor;
166
167
168
169
170 private final ExecutorService appendExecutor;
171
172
173
174
175
176
177
178 private final RingBufferEventHandler ringBufferEventHandler;
179
180
181
182
183
184
185
186 private final Map<Thread, SyncFuture> syncFuturesByHandler;
187
188
189
190
191
192 private volatile long highestUnsyncedSequence = -1;
193
194
195
196
197
198
199 private final AtomicLong highestSyncedSequence = new AtomicLong(0);
200
201
202
203
204 protected final FileSystem fs;
205
206
207
208
209 private final Path fullPathLogDir;
210
211
212
213 private final Path fullPathArchiveDir;
214
215
216
217
218 private final PathFilter ourFiles;
219
220
221
222
223 private final String logFilePrefix;
224
225
226
227
228 private final String logFileSuffix;
229
230
231
232
233 private final String prefixPathStr;
234
235 private final WALCoprocessorHost coprocessorHost;
236
237
238
239
240 protected final Configuration conf;
241
242 private final List<WALActionsListener> listeners = new CopyOnWriteArrayList<WALActionsListener>();
243
244 @Override
245 public void registerWALActionsListener(final WALActionsListener listener) {
246 this.listeners.add(listener);
247 }
248
249 @Override
250 public boolean unregisterWALActionsListener(final WALActionsListener listener) {
251 return this.listeners.remove(listener);
252 }
253
254 @Override
255 public WALCoprocessorHost getCoprocessorHost() {
256 return coprocessorHost;
257 }
258
259
260
261 private FSDataOutputStream hdfs_out;
262
263
264
265
266 private final int minTolerableReplication;
267
268
269 private final Method getNumCurrentReplicas;
270 private final Method getPipeLine;
271 private final int slowSyncNs;
272
273 private final static Object [] NO_ARGS = new Object []{};
274
275
276
277
278
279 private final AtomicInteger consecutiveLogRolls = new AtomicInteger(0);
280
281 private final int lowReplicationRollLimit;
282
283
284
285
286 private volatile boolean lowReplicationRollEnabled = true;
287
288
289
290
291 volatile Writer writer;
292
293
294 private final DrainBarrier closeBarrier = new DrainBarrier();
295
296
297
298
299
300
301
302
303 private final ReentrantLock rollWriterLock = new ReentrantLock(true);
304
305 private volatile boolean closed = false;
306 private final AtomicBoolean shutdown = new AtomicBoolean(false);
307
308
309 private final AtomicLong filenum = new AtomicLong(-1);
310
311
312 private final AtomicInteger numEntries = new AtomicInteger(0);
313
314
315 private final long logrollsize;
316
317
318
319
320 private AtomicLong totalLogSize = new AtomicLong(0);
321
322
323
324
325
326
327 private final int maxLogs;
328
329
330 private final int closeErrorsTolerated;
331
332 private final AtomicInteger closeErrorCount = new AtomicInteger();
333
334
335
336
337
338
339
340
341
342
343
344 private final Object regionSequenceIdLock = new Object();
345
346
347
348
349
350
351
352 private final ConcurrentSkipListMap<byte [], Long> oldestUnflushedRegionSequenceIds =
353 new ConcurrentSkipListMap<byte [], Long>(Bytes.BYTES_COMPARATOR);
354
355
356
357
358
359
360
361
362
363 private final Map<byte[], Long> lowestFlushingRegionSequenceIds =
364 new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
365
366
367
368
369
370
371
372
373
374
375
376
377 private Map<byte[], Long> highestRegionSequenceIds = new HashMap<byte[], Long>();
378
379
380
381
382
383 public final Comparator<Path> LOG_NAME_COMPARATOR = new Comparator<Path>() {
384 @Override
385 public int compare(Path o1, Path o2) {
386 long t1 = getFileNumFromFileName(o1);
387 long t2 = getFileNumFromFileName(o2);
388 if (t1 == t2) return 0;
389 return (t1 > t2) ? 1 : -1;
390 }
391 };
392
393
394
395
396
397 private NavigableMap<Path, Map<byte[], Long>> byWalRegionSequenceIds =
398 new ConcurrentSkipListMap<Path, Map<byte[], Long>>(LOG_NAME_COMPARATOR);
399
400
401
402
403
404 static class RingBufferExceptionHandler implements ExceptionHandler {
405 @Override
406 public void handleEventException(Throwable ex, long sequence, Object event) {
407 LOG.error("Sequence=" + sequence + ", event=" + event, ex);
408 throw new RuntimeException(ex);
409 }
410
411 @Override
412 public void handleOnStartException(Throwable ex) {
413 LOG.error(ex);
414 throw new RuntimeException(ex);
415 }
416
417 @Override
418 public void handleOnShutdownException(Throwable ex) {
419 LOG.error(ex);
420 throw new RuntimeException(ex);
421 }
422 }
423
424
425
426
427
428
429
430
431
432
433 public FSHLog(final FileSystem fs, final Path root, final String logDir, final Configuration conf)
434 throws IOException {
435 this(fs, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, null);
436 }
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462 public FSHLog(final FileSystem fs, final Path rootDir, final String logDir,
463 final String archiveDir, final Configuration conf,
464 final List<WALActionsListener> listeners,
465 final boolean failIfWALExists, final String prefix, final String suffix)
466 throws IOException {
467 this.fs = fs;
468 this.fullPathLogDir = new Path(rootDir, logDir);
469 this.fullPathArchiveDir = new Path(rootDir, archiveDir);
470 this.conf = conf;
471
472 if (!fs.exists(fullPathLogDir) && !fs.mkdirs(fullPathLogDir)) {
473 throw new IOException("Unable to mkdir " + fullPathLogDir);
474 }
475
476 if (!fs.exists(this.fullPathArchiveDir)) {
477 if (!fs.mkdirs(this.fullPathArchiveDir)) {
478 throw new IOException("Unable to mkdir " + this.fullPathArchiveDir);
479 }
480 }
481
482
483 this.logFilePrefix =
484 prefix == null || prefix.isEmpty() ? "wal" : URLEncoder.encode(prefix, "UTF8");
485
486 if (suffix != null && !(suffix.isEmpty()) && !(suffix.startsWith(WAL_FILE_NAME_DELIMITER))) {
487 throw new IllegalArgumentException("wal suffix must start with '" + WAL_FILE_NAME_DELIMITER +
488 "' but instead was '" + suffix + "'");
489 }
490 this.logFileSuffix = (suffix == null) ? "" : URLEncoder.encode(suffix, "UTF8");
491 this.prefixPathStr = new Path(fullPathLogDir,
492 logFilePrefix + WAL_FILE_NAME_DELIMITER).toString();
493
494 this.ourFiles = new PathFilter() {
495 @Override
496 public boolean accept(final Path fileName) {
497
498 final String fileNameString = fileName.toString();
499 if (!fileNameString.startsWith(prefixPathStr)) {
500 return false;
501 }
502 if (logFileSuffix.isEmpty()) {
503
504 return org.apache.commons.lang.StringUtils.isNumeric(
505 fileNameString.substring(prefixPathStr.length()));
506 } else if (!fileNameString.endsWith(logFileSuffix)) {
507 return false;
508 }
509 return true;
510 }
511 };
512
513 if (failIfWALExists) {
514 final FileStatus[] walFiles = FSUtils.listStatus(fs, fullPathLogDir, ourFiles);
515 if (null != walFiles && 0 != walFiles.length) {
516 throw new IOException("Target WAL already exists within directory " + fullPathLogDir);
517 }
518 }
519
520
521 if (listeners != null) {
522 for (WALActionsListener i: listeners) {
523 registerWALActionsListener(i);
524 }
525 }
526 this.coprocessorHost = new WALCoprocessorHost(this, conf);
527
528
529
530 final long blocksize = this.conf.getLong("hbase.regionserver.hlog.blocksize",
531 FSUtils.getDefaultBlockSize(this.fs, this.fullPathLogDir));
532 this.logrollsize =
533 (long)(blocksize * conf.getFloat("hbase.regionserver.logroll.multiplier", 0.95f));
534
535 this.maxLogs = conf.getInt("hbase.regionserver.maxlogs", 32);
536 this.minTolerableReplication = conf.getInt( "hbase.regionserver.hlog.tolerable.lowreplication",
537 FSUtils.getDefaultReplication(fs, this.fullPathLogDir));
538 this.lowReplicationRollLimit =
539 conf.getInt("hbase.regionserver.hlog.lowreplication.rolllimit", 5);
540 this.closeErrorsTolerated = conf.getInt("hbase.regionserver.logroll.errors.tolerated", 0);
541 int maxHandlersCount = conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT, 200);
542
543 LOG.info("WAL configuration: blocksize=" + StringUtils.byteDesc(blocksize) +
544 ", rollsize=" + StringUtils.byteDesc(this.logrollsize) +
545 ", prefix=" + this.logFilePrefix + ", suffix=" + logFileSuffix + ", logDir=" +
546 this.fullPathLogDir + ", archiveDir=" + this.fullPathArchiveDir);
547
548
549 rollWriter();
550
551 this.slowSyncNs =
552 1000000 * conf.getInt("hbase.regionserver.hlog.slowsync.ms",
553 DEFAULT_SLOW_SYNC_TIME_MS);
554
555
556 this.getNumCurrentReplicas = getGetNumCurrentReplicas(this.hdfs_out);
557 this.getPipeLine = getGetPipeline(this.hdfs_out);
558
559
560
561 String hostingThreadName = Thread.currentThread().getName();
562 this.appendExecutor = Executors.
563 newSingleThreadExecutor(Threads.getNamedThreadFactory(hostingThreadName + ".append"));
564
565
566
567
568 final int preallocatedEventCount =
569 this.conf.getInt("hbase.regionserver.wal.disruptor.event.count", 1024 * 16);
570
571
572 this.disruptor =
573 new Disruptor<RingBufferTruck>(RingBufferTruck.EVENT_FACTORY, preallocatedEventCount,
574 this.appendExecutor, ProducerType.MULTI, new BlockingWaitStrategy());
575
576
577 this.disruptor.getRingBuffer().next();
578 this.ringBufferEventHandler =
579 new RingBufferEventHandler(conf.getInt("hbase.regionserver.hlog.syncer.count", 5),
580 maxHandlersCount);
581 this.disruptor.handleExceptionsWith(new RingBufferExceptionHandler());
582 this.disruptor.handleEventsWith(new RingBufferEventHandler [] {this.ringBufferEventHandler});
583
584 this.syncFuturesByHandler = new ConcurrentHashMap<Thread, SyncFuture>(maxHandlersCount);
585
586 this.disruptor.start();
587 }
588
589
590
591
592
593 protected FileStatus[] getFiles() throws IOException {
594 return FSUtils.listStatus(fs, fullPathLogDir, ourFiles);
595 }
596
597
598
599
600
601
602
603
604
605 @VisibleForTesting
606 OutputStream getOutputStream() {
607 return this.hdfs_out.getWrappedStream();
608 }
609
610 @Override
611 public byte [][] rollWriter() throws FailedLogCloseException, IOException {
612 return rollWriter(false);
613 }
614
615
616
617
618
619 private Path getNewPath() throws IOException {
620 this.filenum.set(System.currentTimeMillis());
621 Path newPath = getCurrentFileName();
622 while (fs.exists(newPath)) {
623 this.filenum.incrementAndGet();
624 newPath = getCurrentFileName();
625 }
626 return newPath;
627 }
628
629 Path getOldPath() {
630 long currentFilenum = this.filenum.get();
631 Path oldPath = null;
632 if (currentFilenum > 0) {
633
634 oldPath = computeFilename(currentFilenum);
635 }
636 return oldPath;
637 }
638
639
640
641
642
643 private void tellListenersAboutPreLogRoll(final Path oldPath, final Path newPath)
644 throws IOException {
645 if (!this.listeners.isEmpty()) {
646 for (WALActionsListener i : this.listeners) {
647 i.preLogRoll(oldPath, newPath);
648 }
649 }
650 }
651
652
653
654
655
656 private void tellListenersAboutPostLogRoll(final Path oldPath, final Path newPath)
657 throws IOException {
658 if (!this.listeners.isEmpty()) {
659 for (WALActionsListener i : this.listeners) {
660 i.postLogRoll(oldPath, newPath);
661 }
662 }
663 }
664
665
666
667
668
669
670 private void preemptiveSync(final ProtobufLogWriter nextWriter) {
671 long startTimeNanos = System.nanoTime();
672 try {
673 nextWriter.sync();
674 postSync(System.nanoTime() - startTimeNanos, 0);
675 } catch (IOException e) {
676
677 LOG.warn("pre-sync failed but an optimization so keep going", e);
678 }
679 }
680
681 @Override
682 public byte [][] rollWriter(boolean force) throws FailedLogCloseException, IOException {
683 rollWriterLock.lock();
684 try {
685
686 if (!force && (this.writer != null && this.numEntries.get() <= 0)) return null;
687 byte [][] regionsToFlush = null;
688 if (this.closed) {
689 LOG.debug("WAL closed. Skipping rolling of writer");
690 return regionsToFlush;
691 }
692 if (!closeBarrier.beginOp()) {
693 LOG.debug("WAL closing. Skipping rolling of writer");
694 return regionsToFlush;
695 }
696 TraceScope scope = Trace.startSpan("FSHLog.rollWriter");
697 try {
698 Path oldPath = getOldPath();
699 Path newPath = getNewPath();
700
701 Writer nextWriter = this.createWriterInstance(newPath);
702 FSDataOutputStream nextHdfsOut = null;
703 if (nextWriter instanceof ProtobufLogWriter) {
704 nextHdfsOut = ((ProtobufLogWriter)nextWriter).getStream();
705
706
707 preemptiveSync((ProtobufLogWriter)nextWriter);
708 }
709 tellListenersAboutPreLogRoll(oldPath, newPath);
710
711 newPath = replaceWriter(oldPath, newPath, nextWriter, nextHdfsOut);
712 tellListenersAboutPostLogRoll(oldPath, newPath);
713
714 if (getNumRolledLogFiles() > 0) {
715 cleanOldLogs();
716 regionsToFlush = findRegionsToForceFlush();
717 }
718 } finally {
719 closeBarrier.endOp();
720 assert scope == NullScope.INSTANCE || !scope.isDetached();
721 scope.close();
722 }
723 return regionsToFlush;
724 } finally {
725 rollWriterLock.unlock();
726 }
727 }
728
729
730
731
732
733
734
735 protected Writer createWriterInstance(final Path path) throws IOException {
736 return DefaultWALProvider.createWriter(conf, fs, path, false);
737 }
738
739
740
741
742
743
744
745
746
747
748
749
750 private void cleanOldLogs() throws IOException {
751 Map<byte[], Long> oldestFlushingSeqNumsLocal = null;
752 Map<byte[], Long> oldestUnflushedSeqNumsLocal = null;
753 List<Path> logsToArchive = new ArrayList<Path>();
754
755 synchronized (regionSequenceIdLock) {
756 oldestFlushingSeqNumsLocal = new HashMap<byte[], Long>(this.lowestFlushingRegionSequenceIds);
757 oldestUnflushedSeqNumsLocal =
758 new HashMap<byte[], Long>(this.oldestUnflushedRegionSequenceIds);
759 }
760 for (Map.Entry<Path, Map<byte[], Long>> e : byWalRegionSequenceIds.entrySet()) {
761
762 Path log = e.getKey();
763 Map<byte[], Long> sequenceNums = e.getValue();
764
765 if (areAllRegionsFlushed(sequenceNums, oldestFlushingSeqNumsLocal,
766 oldestUnflushedSeqNumsLocal)) {
767 logsToArchive.add(log);
768 LOG.debug("WAL file ready for archiving " + log);
769 }
770 }
771 for (Path p : logsToArchive) {
772 this.totalLogSize.addAndGet(-this.fs.getFileStatus(p).getLen());
773 archiveLogFile(p);
774 this.byWalRegionSequenceIds.remove(p);
775 }
776 }
777
778
779
780
781
782
783
784
785
786
787
788
789 static boolean areAllRegionsFlushed(Map<byte[], Long> sequenceNums,
790 Map<byte[], Long> oldestFlushingMap, Map<byte[], Long> oldestUnflushedMap) {
791 for (Map.Entry<byte[], Long> regionSeqIdEntry : sequenceNums.entrySet()) {
792
793
794 long oldestFlushing = oldestFlushingMap.containsKey(regionSeqIdEntry.getKey()) ?
795 oldestFlushingMap.get(regionSeqIdEntry.getKey()) : Long.MAX_VALUE;
796 long oldestUnFlushed = oldestUnflushedMap.containsKey(regionSeqIdEntry.getKey()) ?
797 oldestUnflushedMap.get(regionSeqIdEntry.getKey()) : Long.MAX_VALUE;
798
799 long minSeqNum = Math.min(oldestFlushing, oldestUnFlushed);
800 if (minSeqNum <= regionSeqIdEntry.getValue()) return false;
801 }
802 return true;
803 }
804
805
806
807
808
809
810
811
812
813
814
815
816 private byte[][] findEligibleMemstoresToFlush(Map<byte[], Long> regionsSequenceNums) {
817 List<byte[]> regionsToFlush = null;
818
819 synchronized (regionSequenceIdLock) {
820 for (Map.Entry<byte[], Long> e : regionsSequenceNums.entrySet()) {
821 Long unFlushedVal = this.oldestUnflushedRegionSequenceIds.get(e.getKey());
822 if (unFlushedVal != null && unFlushedVal <= e.getValue()) {
823 if (regionsToFlush == null) regionsToFlush = new ArrayList<byte[]>();
824 regionsToFlush.add(e.getKey());
825 }
826 }
827 }
828 return regionsToFlush == null ? null : regionsToFlush
829 .toArray(new byte[][] { HConstants.EMPTY_BYTE_ARRAY });
830 }
831
832
833
834
835
836
837
838
839 byte[][] findRegionsToForceFlush() throws IOException {
840 byte [][] regions = null;
841 int logCount = getNumRolledLogFiles();
842 if (logCount > this.maxLogs && logCount > 0) {
843 Map.Entry<Path, Map<byte[], Long>> firstWALEntry =
844 this.byWalRegionSequenceIds.firstEntry();
845 regions = findEligibleMemstoresToFlush(firstWALEntry.getValue());
846 }
847 if (regions != null) {
848 StringBuilder sb = new StringBuilder();
849 for (int i = 0; i < regions.length; i++) {
850 if (i > 0) sb.append(", ");
851 sb.append(Bytes.toStringBinary(regions[i]));
852 }
853 LOG.info("Too many wals: logs=" + logCount + ", maxlogs=" +
854 this.maxLogs + "; forcing flush of " + regions.length + " regions(s): " +
855 sb.toString());
856 }
857 return regions;
858 }
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878 Path replaceWriter(final Path oldPath, final Path newPath, Writer nextWriter,
879 final FSDataOutputStream nextHdfsOut)
880 throws IOException {
881
882
883
884
885
886 SyncFuture syncFuture = null;
887 SafePointZigZagLatch zigzagLatch = (this.ringBufferEventHandler == null)?
888 null: this.ringBufferEventHandler.attainSafePoint();
889 TraceScope scope = Trace.startSpan("FSHFile.replaceWriter");
890 try {
891
892
893
894
895
896 try {
897 if (zigzagLatch != null) {
898 Trace.addTimelineAnnotation("awaiting safepoint");
899 syncFuture = zigzagLatch.waitSafePoint(publishSyncOnRingBuffer());
900 }
901 } catch (FailedSyncBeforeLogCloseException e) {
902 if (isUnflushedEntries()) throw e;
903
904 LOG.warn("Failed last sync but no outstanding unsync edits so falling through to close; " +
905 e.getMessage());
906 }
907
908
909
910 try {
911 if (this.writer != null) {
912 Trace.addTimelineAnnotation("closing writer");
913 this.writer.close();
914 Trace.addTimelineAnnotation("writer closed");
915 }
916 this.closeErrorCount.set(0);
917 } catch (IOException ioe) {
918 int errors = closeErrorCount.incrementAndGet();
919 if (!isUnflushedEntries() && (errors <= this.closeErrorsTolerated)) {
920 LOG.warn("Riding over failed WAL close of " + oldPath + ", cause=\"" +
921 ioe.getMessage() + "\", errors=" + errors +
922 "; THIS FILE WAS NOT CLOSED BUT ALL EDITS SYNCED SO SHOULD BE OK");
923 } else {
924 throw ioe;
925 }
926 }
927 this.writer = nextWriter;
928 this.hdfs_out = nextHdfsOut;
929 int oldNumEntries = this.numEntries.get();
930 this.numEntries.set(0);
931 final String newPathString = (null == newPath ? null : FSUtils.getPath(newPath));
932 if (oldPath != null) {
933 this.byWalRegionSequenceIds.put(oldPath, this.highestRegionSequenceIds);
934 this.highestRegionSequenceIds = new HashMap<byte[], Long>();
935 long oldFileLen = this.fs.getFileStatus(oldPath).getLen();
936 this.totalLogSize.addAndGet(oldFileLen);
937 LOG.info("Rolled WAL " + FSUtils.getPath(oldPath) + " with entries=" + oldNumEntries +
938 ", filesize=" + StringUtils.byteDesc(oldFileLen) + "; new WAL " +
939 newPathString);
940 } else {
941 LOG.info("New WAL " + newPathString);
942 }
943 } catch (InterruptedException ie) {
944
945 Thread.currentThread().interrupt();
946 } catch (IOException e) {
947 long count = getUnflushedEntriesCount();
948 LOG.error("Failed close of WAL writer " + oldPath + ", unflushedEntries=" + count, e);
949 throw new FailedLogCloseException(oldPath + ", unflushedEntries=" + count, e);
950 } finally {
951 try {
952
953 if (zigzagLatch != null) {
954 zigzagLatch.releaseSafePoint();
955
956 if (syncFuture != null) blockOnSync(syncFuture);
957 }
958 } finally {
959 scope.close();
960 }
961 }
962 return newPath;
963 }
964
965 long getUnflushedEntriesCount() {
966 long highestSynced = this.highestSyncedSequence.get();
967 return highestSynced > this.highestUnsyncedSequence?
968 0: this.highestUnsyncedSequence - highestSynced;
969 }
970
971 boolean isUnflushedEntries() {
972 return getUnflushedEntriesCount() > 0;
973 }
974
975
976
977
978
979 public static Path getWALArchivePath(Path archiveDir, Path p) {
980 return new Path(archiveDir, p.getName());
981 }
982
983 private void archiveLogFile(final Path p) throws IOException {
984 Path newPath = getWALArchivePath(this.fullPathArchiveDir, p);
985
986 if (!this.listeners.isEmpty()) {
987 for (WALActionsListener i : this.listeners) {
988 i.preLogArchive(p, newPath);
989 }
990 }
991 LOG.info("Archiving " + p + " to " + newPath);
992 if (!FSUtils.renameAndSetModifyTime(this.fs, p, newPath)) {
993 throw new IOException("Unable to rename " + p + " to " + newPath);
994 }
995
996 if (!this.listeners.isEmpty()) {
997 for (WALActionsListener i : this.listeners) {
998 i.postLogArchive(p, newPath);
999 }
1000 }
1001 }
1002
1003
1004
1005
1006
1007
1008
1009 protected Path computeFilename(final long filenum) {
1010 if (filenum < 0) {
1011 throw new RuntimeException("wal file number can't be < 0");
1012 }
1013 String child = logFilePrefix + WAL_FILE_NAME_DELIMITER + filenum + logFileSuffix;
1014 return new Path(fullPathLogDir, child);
1015 }
1016
1017
1018
1019
1020
1021
1022 public Path getCurrentFileName() {
1023 return computeFilename(this.filenum.get());
1024 }
1025
1026 @Override
1027 public String toString() {
1028 return "FSHLog " + logFilePrefix + ":" + logFileSuffix + "(num " + filenum + ")";
1029 }
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039 protected long getFileNumFromFileName(Path fileName) {
1040 if (fileName == null) throw new IllegalArgumentException("file name can't be null");
1041 if (!ourFiles.accept(fileName)) {
1042 throw new IllegalArgumentException("The log file " + fileName +
1043 " doesn't belong to this wal. (" + toString() + ")");
1044 }
1045 final String fileNameString = fileName.toString();
1046 String chompedPath = fileNameString.substring(prefixPathStr.length(),
1047 (fileNameString.length() - logFileSuffix.length()));
1048 return Long.parseLong(chompedPath);
1049 }
1050
1051 @Override
1052 public void close() throws IOException {
1053 shutdown();
1054 final FileStatus[] files = getFiles();
1055 if (null != files && 0 != files.length) {
1056 for (FileStatus file : files) {
1057 Path p = getWALArchivePath(this.fullPathArchiveDir, file.getPath());
1058
1059 if (!this.listeners.isEmpty()) {
1060 for (WALActionsListener i : this.listeners) {
1061 i.preLogArchive(file.getPath(), p);
1062 }
1063 }
1064
1065 if (!FSUtils.renameAndSetModifyTime(fs, file.getPath(), p)) {
1066 throw new IOException("Unable to rename " + file.getPath() + " to " + p);
1067 }
1068
1069 if (!this.listeners.isEmpty()) {
1070 for (WALActionsListener i : this.listeners) {
1071 i.postLogArchive(file.getPath(), p);
1072 }
1073 }
1074 }
1075 LOG.debug("Moved " + files.length + " WAL file(s) to " +
1076 FSUtils.getPath(this.fullPathArchiveDir));
1077 }
1078 LOG.info("Closed WAL: " + toString() );
1079 }
1080
1081 @Override
1082 public void shutdown() throws IOException {
1083 if (shutdown.compareAndSet(false, true)) {
1084 try {
1085
1086 closeBarrier.stopAndDrainOps();
1087 } catch (InterruptedException e) {
1088 LOG.error("Exception while waiting for cache flushes and log rolls", e);
1089 Thread.currentThread().interrupt();
1090 }
1091
1092
1093
1094
1095 if (this.disruptor != null) {
1096 long timeoutms = conf.getLong("hbase.wal.disruptor.shutdown.timeout.ms", 60000);
1097 try {
1098 this.disruptor.shutdown(timeoutms, TimeUnit.MILLISECONDS);
1099 } catch (TimeoutException e) {
1100 LOG.warn("Timed out bringing down disruptor after " + timeoutms + "ms; forcing halt " +
1101 "(It is a problem if this is NOT an ABORT! -- DATALOSS!!!!)");
1102 this.disruptor.halt();
1103 this.disruptor.shutdown();
1104 }
1105 }
1106
1107 if (this.appendExecutor != null) this.appendExecutor.shutdown();
1108
1109
1110 if (!this.listeners.isEmpty()) {
1111 for (WALActionsListener i : this.listeners) {
1112 i.logCloseRequested();
1113 }
1114 }
1115 this.closed = true;
1116 if (LOG.isDebugEnabled()) {
1117 LOG.debug("Closing WAL writer in " + FSUtils.getPath(fullPathLogDir));
1118 }
1119 if (this.writer != null) {
1120 this.writer.close();
1121 this.writer = null;
1122 }
1123 }
1124 }
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134 protected WALKey makeKey(byte[] encodedRegionName, TableName tableName, long seqnum,
1135 long now, List<UUID> clusterIds, long nonceGroup, long nonce) {
1136
1137 return new HLogKey(encodedRegionName, tableName, seqnum, now, clusterIds, nonceGroup, nonce);
1138 }
1139
1140 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NP_NULL_ON_SOME_PATH_EXCEPTION",
1141 justification="Will never be null")
1142 @Override
1143 public long append(final HTableDescriptor htd, final HRegionInfo hri, final WALKey key,
1144 final WALEdit edits, final AtomicLong sequenceId, final boolean inMemstore,
1145 final List<Cell> memstoreCells) throws IOException {
1146 if (this.closed) throw new IOException("Cannot append; log is closed");
1147
1148
1149 TraceScope scope = Trace.startSpan("FSHLog.append");
1150
1151
1152
1153
1154 FSWALEntry entry = null;
1155 long sequence = this.disruptor.getRingBuffer().next();
1156 try {
1157 RingBufferTruck truck = this.disruptor.getRingBuffer().get(sequence);
1158
1159
1160
1161 entry = new FSWALEntry(sequence, key, edits, sequenceId, inMemstore, htd, hri, memstoreCells);
1162 truck.loadPayload(entry, scope.detach());
1163 } finally {
1164 this.disruptor.getRingBuffer().publish(sequence);
1165 }
1166 return sequence;
1167 }
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184 private class SyncRunner extends HasThread {
1185 private volatile long sequence;
1186 private final BlockingQueue<SyncFuture> syncFutures;
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198 SyncRunner(final String name, final int maxHandlersCount) {
1199 super(name);
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213 this.syncFutures = new LinkedBlockingQueue<SyncFuture>(maxHandlersCount * 3);
1214 }
1215
1216 void offer(final long sequence, final SyncFuture [] syncFutures, final int syncFutureCount) {
1217
1218 this.sequence = sequence;
1219 this.syncFutures.addAll(Arrays.asList(syncFutures).subList(0, syncFutureCount));
1220 }
1221
1222
1223
1224
1225
1226
1227
1228
1229 private int releaseSyncFuture(final SyncFuture syncFuture, final long currentSequence,
1230 final Throwable t) {
1231 if (!syncFuture.done(currentSequence, t)) throw new IllegalStateException();
1232
1233 return 1;
1234 }
1235
1236
1237
1238
1239
1240
1241
1242 private int releaseSyncFutures(final long currentSequence, final Throwable t) {
1243 int syncCount = 0;
1244 for (SyncFuture syncFuture; (syncFuture = this.syncFutures.peek()) != null;) {
1245 if (syncFuture.getRingBufferSequence() > currentSequence) break;
1246 releaseSyncFuture(syncFuture, currentSequence, t);
1247 if (!this.syncFutures.remove(syncFuture)) {
1248 throw new IllegalStateException(syncFuture.toString());
1249 }
1250 syncCount++;
1251 }
1252 return syncCount;
1253 }
1254
1255
1256
1257
1258
1259 private long updateHighestSyncedSequence(long sequence) {
1260 long currentHighestSyncedSequence;
1261
1262 do {
1263 currentHighestSyncedSequence = highestSyncedSequence.get();
1264 if (currentHighestSyncedSequence >= sequence) {
1265
1266
1267 sequence = currentHighestSyncedSequence;
1268 break;
1269 }
1270 } while (!highestSyncedSequence.compareAndSet(currentHighestSyncedSequence, sequence));
1271 return sequence;
1272 }
1273
1274 public void run() {
1275 long currentSequence;
1276 while (!isInterrupted()) {
1277 int syncCount = 0;
1278 SyncFuture takeSyncFuture;
1279 try {
1280 while (true) {
1281
1282 takeSyncFuture = this.syncFutures.take();
1283 currentSequence = this.sequence;
1284 long syncFutureSequence = takeSyncFuture.getRingBufferSequence();
1285 if (syncFutureSequence > currentSequence) {
1286 throw new IllegalStateException("currentSequence=" + syncFutureSequence +
1287 ", syncFutureSequence=" + syncFutureSequence);
1288 }
1289
1290 long currentHighestSyncedSequence = highestSyncedSequence.get();
1291 if (currentSequence < currentHighestSyncedSequence) {
1292 syncCount += releaseSyncFuture(takeSyncFuture, currentHighestSyncedSequence, null);
1293
1294 continue;
1295 }
1296 break;
1297 }
1298
1299
1300 TraceScope scope = Trace.continueSpan(takeSyncFuture.getSpan());
1301 long start = System.nanoTime();
1302 Throwable t = null;
1303 try {
1304 Trace.addTimelineAnnotation("syncing writer");
1305 writer.sync();
1306 Trace.addTimelineAnnotation("writer synced");
1307 currentSequence = updateHighestSyncedSequence(currentSequence);
1308 } catch (IOException e) {
1309 LOG.error("Error syncing, request close of wal ", e);
1310 t = e;
1311 } catch (Exception e) {
1312 LOG.warn("UNEXPECTED", e);
1313 t = e;
1314 } finally {
1315
1316 takeSyncFuture.setSpan(scope.detach());
1317
1318 syncCount += releaseSyncFuture(takeSyncFuture, currentSequence, t);
1319
1320 syncCount += releaseSyncFutures(currentSequence, t);
1321 if (t != null) {
1322 requestLogRoll();
1323 } else checkLogRoll();
1324 }
1325 postSync(System.nanoTime() - start, syncCount);
1326 } catch (InterruptedException e) {
1327
1328 Thread.currentThread().interrupt();
1329 } catch (Throwable t) {
1330 LOG.warn("UNEXPECTED, continuing", t);
1331 }
1332 }
1333 }
1334 }
1335
1336
1337
1338
1339 void checkLogRoll() {
1340
1341 if (!rollWriterLock.tryLock()) return;
1342 boolean lowReplication;
1343 try {
1344 lowReplication = checkLowReplication();
1345 } finally {
1346 rollWriterLock.unlock();
1347 }
1348 try {
1349 if (lowReplication || writer != null && writer.getLength() > logrollsize) {
1350 requestLogRoll(lowReplication);
1351 }
1352 } catch (IOException e) {
1353 LOG.warn("Writer.getLength() failed; continuing", e);
1354 }
1355 }
1356
1357
1358
1359
1360 private boolean checkLowReplication() {
1361 boolean logRollNeeded = false;
1362
1363
1364 try {
1365 int numCurrentReplicas = getLogReplication();
1366 if (numCurrentReplicas != 0 && numCurrentReplicas < this.minTolerableReplication) {
1367 if (this.lowReplicationRollEnabled) {
1368 if (this.consecutiveLogRolls.get() < this.lowReplicationRollLimit) {
1369 LOG.warn("HDFS pipeline error detected. " + "Found "
1370 + numCurrentReplicas + " replicas but expecting no less than "
1371 + this.minTolerableReplication + " replicas. "
1372 + " Requesting close of wal. current pipeline: "
1373 + Arrays.toString(getPipeLine()));
1374 logRollNeeded = true;
1375
1376
1377
1378 this.consecutiveLogRolls.getAndIncrement();
1379 } else {
1380 LOG.warn("Too many consecutive RollWriter requests, it's a sign of "
1381 + "the total number of live datanodes is lower than the tolerable replicas.");
1382 this.consecutiveLogRolls.set(0);
1383 this.lowReplicationRollEnabled = false;
1384 }
1385 }
1386 } else if (numCurrentReplicas >= this.minTolerableReplication) {
1387 if (!this.lowReplicationRollEnabled) {
1388
1389
1390
1391 if (this.numEntries.get() <= 1) {
1392 return logRollNeeded;
1393 }
1394
1395
1396 this.lowReplicationRollEnabled = true;
1397 LOG.info("LowReplication-Roller was enabled.");
1398 }
1399 }
1400 } catch (Exception e) {
1401 LOG.warn("Unable to invoke DFSOutputStream.getNumCurrentReplicas" + e +
1402 " still proceeding ahead...");
1403 }
1404 return logRollNeeded;
1405 }
1406
1407 private SyncFuture publishSyncOnRingBuffer() {
1408 return publishSyncOnRingBuffer(null);
1409 }
1410
1411 private SyncFuture publishSyncOnRingBuffer(Span span) {
1412 long sequence = this.disruptor.getRingBuffer().next();
1413 SyncFuture syncFuture = getSyncFuture(sequence, span);
1414 try {
1415 RingBufferTruck truck = this.disruptor.getRingBuffer().get(sequence);
1416 truck.loadPayload(syncFuture);
1417 } finally {
1418 this.disruptor.getRingBuffer().publish(sequence);
1419 }
1420 return syncFuture;
1421 }
1422
1423
1424 private Span publishSyncThenBlockOnCompletion(Span span) throws IOException {
1425 return blockOnSync(publishSyncOnRingBuffer(span));
1426 }
1427
1428 private Span blockOnSync(final SyncFuture syncFuture) throws IOException {
1429
1430 try {
1431 syncFuture.get();
1432 return syncFuture.getSpan();
1433 } catch (InterruptedException ie) {
1434 LOG.warn("Interrupted", ie);
1435 throw convertInterruptedExceptionToIOException(ie);
1436 } catch (ExecutionException e) {
1437 throw ensureIOException(e.getCause());
1438 }
1439 }
1440
1441 private IOException convertInterruptedExceptionToIOException(final InterruptedException ie) {
1442 Thread.currentThread().interrupt();
1443 IOException ioe = new InterruptedIOException();
1444 ioe.initCause(ie);
1445 return ioe;
1446 }
1447
1448 private SyncFuture getSyncFuture(final long sequence, Span span) {
1449 SyncFuture syncFuture = this.syncFuturesByHandler.get(Thread.currentThread());
1450 if (syncFuture == null) {
1451 syncFuture = new SyncFuture();
1452 this.syncFuturesByHandler.put(Thread.currentThread(), syncFuture);
1453 }
1454 return syncFuture.reset(sequence, span);
1455 }
1456
1457 private void postSync(final long timeInNanos, final int handlerSyncs) {
1458 if (timeInNanos > this.slowSyncNs) {
1459 String msg =
1460 new StringBuilder().append("Slow sync cost: ")
1461 .append(timeInNanos / 1000000).append(" ms, current pipeline: ")
1462 .append(Arrays.toString(getPipeLine())).toString();
1463 Trace.addTimelineAnnotation(msg);
1464 LOG.info(msg);
1465 }
1466 if (!listeners.isEmpty()) {
1467 for (WALActionsListener listener : listeners) {
1468 listener.postSync(timeInNanos, handlerSyncs);
1469 }
1470 }
1471 }
1472
1473 private long postAppend(final Entry e, final long elapsedTime) {
1474 long len = 0;
1475 if (!listeners.isEmpty()) {
1476 for (Cell cell : e.getEdit().getCells()) {
1477 len += CellUtil.estimatedSerializedSizeOf(cell);
1478 }
1479 for (WALActionsListener listener : listeners) {
1480 listener.postAppend(len, elapsedTime);
1481 }
1482 }
1483 return len;
1484 }
1485
1486
1487
1488
1489
1490
1491 private Method getGetNumCurrentReplicas(final FSDataOutputStream os) {
1492
1493
1494 Method m = null;
1495 if (os != null) {
1496 Class<? extends OutputStream> wrappedStreamClass = os.getWrappedStream().getClass();
1497 try {
1498 m = wrappedStreamClass.getDeclaredMethod("getNumCurrentReplicas", new Class<?>[] {});
1499 m.setAccessible(true);
1500 } catch (NoSuchMethodException e) {
1501 LOG.info("FileSystem's output stream doesn't support getNumCurrentReplicas; " +
1502 "HDFS-826 not available; fsOut=" + wrappedStreamClass.getName());
1503 } catch (SecurityException e) {
1504 LOG.info("No access to getNumCurrentReplicas on FileSystems's output stream; HDFS-826 " +
1505 "not available; fsOut=" + wrappedStreamClass.getName(), e);
1506 m = null;
1507 }
1508 }
1509 if (m != null) {
1510 if (LOG.isTraceEnabled()) LOG.trace("Using getNumCurrentReplicas");
1511 }
1512 return m;
1513 }
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527 @VisibleForTesting
1528 int getLogReplication()
1529 throws IllegalArgumentException, IllegalAccessException, InvocationTargetException {
1530 final OutputStream stream = getOutputStream();
1531 if (this.getNumCurrentReplicas != null && stream != null) {
1532 Object repl = this.getNumCurrentReplicas.invoke(stream, NO_ARGS);
1533 if (repl instanceof Integer) {
1534 return ((Integer)repl).intValue();
1535 }
1536 }
1537 return 0;
1538 }
1539
1540 @Override
1541 public void sync() throws IOException {
1542 TraceScope scope = Trace.startSpan("FSHLog.sync");
1543 try {
1544 scope = Trace.continueSpan(publishSyncThenBlockOnCompletion(scope.detach()));
1545 } finally {
1546 assert scope == NullScope.INSTANCE || !scope.isDetached();
1547 scope.close();
1548 }
1549 }
1550
1551 @Override
1552 public void sync(long txid) throws IOException {
1553 if (this.highestSyncedSequence.get() >= txid){
1554
1555 return;
1556 }
1557 TraceScope scope = Trace.startSpan("FSHLog.sync");
1558 try {
1559 scope = Trace.continueSpan(publishSyncThenBlockOnCompletion(scope.detach()));
1560 } finally {
1561 assert scope == NullScope.INSTANCE || !scope.isDetached();
1562 scope.close();
1563 }
1564 }
1565
1566
1567 public void requestLogRoll() {
1568 requestLogRoll(false);
1569 }
1570
1571 private void requestLogRoll(boolean tooFewReplicas) {
1572 if (!this.listeners.isEmpty()) {
1573 for (WALActionsListener i: this.listeners) {
1574 i.logRollRequested(tooFewReplicas);
1575 }
1576 }
1577 }
1578
1579
1580
1581 public int getNumRolledLogFiles() {
1582 return byWalRegionSequenceIds.size();
1583 }
1584
1585
1586
1587 public int getNumLogFiles() {
1588
1589 return getNumRolledLogFiles() + 1;
1590 }
1591
1592
1593
1594 public long getLogFileSize() {
1595 return this.totalLogSize.get();
1596 }
1597
1598 @Override
1599 public boolean startCacheFlush(final byte[] encodedRegionName) {
1600 Long oldRegionSeqNum = null;
1601 if (!closeBarrier.beginOp()) {
1602 LOG.info("Flush will not be started for " + Bytes.toString(encodedRegionName) +
1603 " - because the server is closing.");
1604 return false;
1605 }
1606 synchronized (regionSequenceIdLock) {
1607 oldRegionSeqNum = this.oldestUnflushedRegionSequenceIds.remove(encodedRegionName);
1608 if (oldRegionSeqNum != null) {
1609 Long oldValue =
1610 this.lowestFlushingRegionSequenceIds.put(encodedRegionName, oldRegionSeqNum);
1611 assert oldValue ==
1612 null : "Flushing map not cleaned up for " + Bytes.toString(encodedRegionName);
1613 }
1614 }
1615 if (oldRegionSeqNum == null) {
1616
1617
1618
1619
1620
1621 LOG.warn("Couldn't find oldest seqNum for the region we are about to flush: ["
1622 + Bytes.toString(encodedRegionName) + "]");
1623 }
1624 return true;
1625 }
1626
1627 @Override
1628 public void completeCacheFlush(final byte [] encodedRegionName) {
1629 synchronized (regionSequenceIdLock) {
1630 this.lowestFlushingRegionSequenceIds.remove(encodedRegionName);
1631 }
1632 closeBarrier.endOp();
1633 }
1634
1635 @Override
1636 public void abortCacheFlush(byte[] encodedRegionName) {
1637 Long currentSeqNum = null, seqNumBeforeFlushStarts = null;
1638 synchronized (regionSequenceIdLock) {
1639 seqNumBeforeFlushStarts = this.lowestFlushingRegionSequenceIds.remove(encodedRegionName);
1640 if (seqNumBeforeFlushStarts != null) {
1641 currentSeqNum =
1642 this.oldestUnflushedRegionSequenceIds.put(encodedRegionName, seqNumBeforeFlushStarts);
1643 }
1644 }
1645 closeBarrier.endOp();
1646 if ((currentSeqNum != null)
1647 && (currentSeqNum.longValue() <= seqNumBeforeFlushStarts.longValue())) {
1648 String errorStr = "Region " + Bytes.toString(encodedRegionName) +
1649 "acquired edits out of order current memstore seq=" + currentSeqNum
1650 + ", previous oldest unflushed id=" + seqNumBeforeFlushStarts;
1651 LOG.error(errorStr);
1652 assert false : errorStr;
1653 Runtime.getRuntime().halt(1);
1654 }
1655 }
1656
1657 @VisibleForTesting
1658 boolean isLowReplicationRollEnabled() {
1659 return lowReplicationRollEnabled;
1660 }
1661
1662 public static final long FIXED_OVERHEAD = ClassSize.align(
1663 ClassSize.OBJECT + (5 * ClassSize.REFERENCE) +
1664 ClassSize.ATOMIC_INTEGER + Bytes.SIZEOF_INT + (3 * Bytes.SIZEOF_LONG));
1665
1666 private static void split(final Configuration conf, final Path p)
1667 throws IOException {
1668 FileSystem fs = FileSystem.get(conf);
1669 if (!fs.exists(p)) {
1670 throw new FileNotFoundException(p.toString());
1671 }
1672 if (!fs.getFileStatus(p).isDirectory()) {
1673 throw new IOException(p + " is not a directory");
1674 }
1675
1676 final Path baseDir = FSUtils.getRootDir(conf);
1677 final Path archiveDir = new Path(baseDir, HConstants.HREGION_OLDLOGDIR_NAME);
1678 WALSplitter.split(baseDir, p, archiveDir, fs, conf, WALFactory.getInstance(conf));
1679 }
1680
1681
1682 @Override
1683 public long getEarliestMemstoreSeqNum(byte[] encodedRegionName) {
1684 Long result = oldestUnflushedRegionSequenceIds.get(encodedRegionName);
1685 return result == null ? HConstants.NO_SEQNUM : result.longValue();
1686 }
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714 static class SafePointZigZagLatch {
1715
1716
1717
1718 private volatile CountDownLatch safePointAttainedLatch = new CountDownLatch(1);
1719
1720
1721
1722 private volatile CountDownLatch safePointReleasedLatch = new CountDownLatch(1);
1723
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733
1734 SyncFuture waitSafePoint(final SyncFuture syncFuture)
1735 throws InterruptedException, FailedSyncBeforeLogCloseException {
1736 while (true) {
1737 if (this.safePointAttainedLatch.await(1, TimeUnit.NANOSECONDS)) break;
1738 if (syncFuture.isThrowable()) {
1739 throw new FailedSyncBeforeLogCloseException(syncFuture.getThrowable());
1740 }
1741 }
1742 return syncFuture;
1743 }
1744
1745
1746
1747
1748
1749
1750
1751 void safePointAttained() throws InterruptedException {
1752 this.safePointAttainedLatch.countDown();
1753 this.safePointReleasedLatch.await();
1754 }
1755
1756
1757
1758
1759
1760 void releaseSafePoint() {
1761 this.safePointReleasedLatch.countDown();
1762 }
1763
1764
1765
1766
1767 boolean isCocked() {
1768 return this.safePointAttainedLatch.getCount() > 0 &&
1769 this.safePointReleasedLatch.getCount() > 0;
1770 }
1771 }
1772
1773
1774
1775
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796 class RingBufferEventHandler implements EventHandler<RingBufferTruck>, LifecycleAware {
1797 private final SyncRunner [] syncRunners;
1798 private final SyncFuture [] syncFutures;
1799
1800
1801 private volatile int syncFuturesCount = 0;
1802 private volatile SafePointZigZagLatch zigzagLatch;
1803
1804
1805
1806 private final Object safePointWaiter = new Object();
1807 private volatile boolean shutdown = false;
1808
1809
1810
1811
1812 private int syncRunnerIndex;
1813
1814 RingBufferEventHandler(final int syncRunnerCount, final int maxHandlersCount) {
1815 this.syncFutures = new SyncFuture[maxHandlersCount];
1816 this.syncRunners = new SyncRunner[syncRunnerCount];
1817 for (int i = 0; i < syncRunnerCount; i++) {
1818 this.syncRunners[i] = new SyncRunner("sync." + i, maxHandlersCount);
1819 }
1820 }
1821
1822 private void cleanupOutstandingSyncsOnException(final long sequence, final Exception e) {
1823 for (int i = 0; i < this.syncFuturesCount; i++) this.syncFutures[i].done(sequence, e);
1824 this.syncFuturesCount = 0;
1825 }
1826
1827 @Override
1828
1829 public void onEvent(final RingBufferTruck truck, final long sequence, boolean endOfBatch)
1830 throws Exception {
1831
1832
1833
1834
1835 try {
1836 if (truck.hasSyncFuturePayload()) {
1837 this.syncFutures[this.syncFuturesCount++] = truck.unloadSyncFuturePayload();
1838
1839 if (this.syncFuturesCount == this.syncFutures.length) endOfBatch = true;
1840 } else if (truck.hasFSWALEntryPayload()) {
1841 TraceScope scope = Trace.continueSpan(truck.unloadSpanPayload());
1842 try {
1843 append(truck.unloadFSWALEntryPayload());
1844 } catch (Exception e) {
1845
1846
1847 cleanupOutstandingSyncsOnException(sequence, e);
1848
1849 return;
1850 } finally {
1851 assert scope == NullScope.INSTANCE || !scope.isDetached();
1852 scope.close();
1853 }
1854 } else {
1855
1856 cleanupOutstandingSyncsOnException(sequence,
1857 new IllegalStateException("Neither append nor sync"));
1858
1859 return;
1860 }
1861
1862
1863
1864
1865
1866 if (!endOfBatch || this.syncFuturesCount <= 0) return;
1867
1868
1869
1870 if (LOG.isTraceEnabled()) {
1871 LOG.trace("Sequence=" + sequence + ", syncCount=" + this.syncFuturesCount);
1872 }
1873
1874
1875
1876
1877
1878 int index = Math.abs(this.syncRunnerIndex++) % this.syncRunners.length;
1879 try {
1880 this.syncRunners[index].offer(sequence, this.syncFutures, this.syncFuturesCount);
1881 } catch (Exception e) {
1882 cleanupOutstandingSyncsOnException(sequence, e);
1883 throw e;
1884 }
1885 attainSafePoint(sequence);
1886 this.syncFuturesCount = 0;
1887 } catch (Throwable t) {
1888 LOG.error("UNEXPECTED!!! syncFutures.length=" + this.syncFutures.length, t);
1889 }
1890 }
1891
1892 SafePointZigZagLatch attainSafePoint() {
1893 this.zigzagLatch = new SafePointZigZagLatch();
1894 return this.zigzagLatch;
1895 }
1896
1897
1898
1899
1900
1901 private void attainSafePoint(final long currentSequence) {
1902 if (this.zigzagLatch == null || !this.zigzagLatch.isCocked()) return;
1903
1904 try {
1905
1906
1907 while (!this.shutdown && this.zigzagLatch.isCocked() &&
1908 highestSyncedSequence.get() < currentSequence) {
1909 synchronized (this.safePointWaiter) {
1910 this.safePointWaiter.wait(0, 1);
1911 }
1912 }
1913
1914 this.zigzagLatch.safePointAttained();
1915 } catch (InterruptedException e) {
1916 LOG.warn("Interrupted ", e);
1917 Thread.currentThread().interrupt();
1918 }
1919 }
1920
1921
1922
1923
1924
1925
1926 void append(final FSWALEntry entry) throws Exception {
1927
1928 atHeadOfRingBufferEventHandlerAppend();
1929
1930 long start = EnvironmentEdgeManager.currentTime();
1931 byte [] encodedRegionName = entry.getKey().getEncodedRegionName();
1932 long regionSequenceId = WALKey.NO_SEQUENCE_ID;
1933 try {
1934
1935
1936
1937 regionSequenceId = entry.stampRegionSequenceId();
1938
1939
1940
1941
1942 if (entry.getEdit().isEmpty()) {
1943 return;
1944 }
1945
1946
1947 if (!coprocessorHost.preWALWrite(entry.getHRegionInfo(), entry.getKey(),
1948 entry.getEdit())) {
1949 if (entry.getEdit().isReplay()) {
1950
1951 entry.getKey().setScopes(null);
1952 }
1953 }
1954 if (!listeners.isEmpty()) {
1955 for (WALActionsListener i: listeners) {
1956
1957 i.visitLogEntryBeforeWrite(entry.getHTableDescriptor(), entry.getKey(),
1958 entry.getEdit());
1959 }
1960 }
1961
1962 writer.append(entry);
1963 assert highestUnsyncedSequence < entry.getSequence();
1964 highestUnsyncedSequence = entry.getSequence();
1965 Long lRegionSequenceId = Long.valueOf(regionSequenceId);
1966 highestRegionSequenceIds.put(encodedRegionName, lRegionSequenceId);
1967 if (entry.isInMemstore()) {
1968 oldestUnflushedRegionSequenceIds.putIfAbsent(encodedRegionName, lRegionSequenceId);
1969 }
1970
1971 coprocessorHost.postWALWrite(entry.getHRegionInfo(), entry.getKey(), entry.getEdit());
1972
1973 postAppend(entry, EnvironmentEdgeManager.currentTime() - start);
1974 } catch (Exception e) {
1975 LOG.fatal("Could not append. Requesting close of wal", e);
1976 requestLogRoll();
1977 throw e;
1978 }
1979 numEntries.incrementAndGet();
1980 }
1981
1982 @Override
1983 public void onStart() {
1984 for (SyncRunner syncRunner: this.syncRunners) syncRunner.start();
1985 }
1986
1987 @Override
1988 public void onShutdown() {
1989 for (SyncRunner syncRunner: this.syncRunners) syncRunner.interrupt();
1990 }
1991 }
1992
1993
1994
1995
1996 @VisibleForTesting
1997 void atHeadOfRingBufferEventHandlerAppend() {
1998
1999 }
2000
2001 private static IOException ensureIOException(final Throwable t) {
2002 return (t instanceof IOException)? (IOException)t: new IOException(t);
2003 }
2004
2005 private static void usage() {
2006 System.err.println("Usage: FSHLog <ARGS>");
2007 System.err.println("Arguments:");
2008 System.err.println(" --dump Dump textual representation of passed one or more files");
2009 System.err.println(" For example: " +
2010 "FSHLog --dump hdfs://example.com:9000/hbase/.logs/MACHINE/LOGFILE");
2011 System.err.println(" --split Split the passed directory of WAL logs");
2012 System.err.println(" For example: " +
2013 "FSHLog --split hdfs://example.com:9000/hbase/.logs/DIR");
2014 }
2015
2016
2017
2018
2019
2020
2021
2022
2023 public static void main(String[] args) throws IOException {
2024 if (args.length < 2) {
2025 usage();
2026 System.exit(-1);
2027 }
2028
2029 if (args[0].compareTo("--dump") == 0) {
2030 WALPrettyPrinter.run(Arrays.copyOfRange(args, 1, args.length));
2031 } else if (args[0].compareTo("--perf") == 0) {
2032 LOG.fatal("Please use the WALPerformanceEvaluation tool instead. i.e.:");
2033 LOG.fatal("\thbase org.apache.hadoop.hbase.wal.WALPerformanceEvaluation --iterations " +
2034 args[1]);
2035 System.exit(-1);
2036 } else if (args[0].compareTo("--split") == 0) {
2037 Configuration conf = HBaseConfiguration.create();
2038 for (int i = 1; i < args.length; i++) {
2039 try {
2040 Path logPath = new Path(args[i]);
2041 FSUtils.setFsDefault(conf, logPath);
2042 split(conf, logPath);
2043 } catch (IOException t) {
2044 t.printStackTrace(System.err);
2045 System.exit(-1);
2046 }
2047 }
2048 } else {
2049 usage();
2050 System.exit(-1);
2051 }
2052 }
2053
2054
2055
2056
2057
2058 private Method getGetPipeline(final FSDataOutputStream os) {
2059 Method m = null;
2060 if (os != null) {
2061 Class<? extends OutputStream> wrappedStreamClass = os.getWrappedStream()
2062 .getClass();
2063 try {
2064 m = wrappedStreamClass.getDeclaredMethod("getPipeline",
2065 new Class<?>[] {});
2066 m.setAccessible(true);
2067 } catch (NoSuchMethodException e) {
2068 LOG.info("FileSystem's output stream doesn't support"
2069 + " getPipeline; not available; fsOut="
2070 + wrappedStreamClass.getName());
2071 } catch (SecurityException e) {
2072 LOG.info(
2073 "Doesn't have access to getPipeline on "
2074 + "FileSystems's output stream ; fsOut="
2075 + wrappedStreamClass.getName(), e);
2076 m = null;
2077 }
2078 }
2079 return m;
2080 }
2081
2082
2083
2084
2085 @VisibleForTesting
2086 DatanodeInfo[] getPipeLine() {
2087 if (this.getPipeLine != null && this.hdfs_out != null) {
2088 Object repl;
2089 try {
2090 repl = this.getPipeLine.invoke(getOutputStream(), NO_ARGS);
2091 if (repl instanceof DatanodeInfo[]) {
2092 return ((DatanodeInfo[]) repl);
2093 }
2094 } catch (Exception e) {
2095 LOG.info("Get pipeline failed", e);
2096 }
2097 }
2098 return new DatanodeInfo[0];
2099 }
2100 }