1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.regionserver.wal;
21
22 import java.io.IOException;
23
24 import org.apache.hadoop.classification.InterfaceAudience;
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.fs.FSDataInputStream;
27 import org.apache.hadoop.fs.FileSystem;
28 import org.apache.hadoop.fs.Path;
29 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
30 import org.apache.hadoop.hbase.util.FSUtils;
31
32 @InterfaceAudience.Private
33 public abstract class ReaderBase implements HLog.Reader {
34 protected Configuration conf;
35 protected FileSystem fs;
36 protected Path path;
37 protected long edit = 0;
38 protected long fileLength;
39 protected WALTrailer trailer;
40
41
42 protected int trailerWarnSize;
43
44
45
46 protected CompressionContext compressionContext = null;
47 protected boolean emptyCompressionContext = true;
48
49
50
51
52 public ReaderBase() {
53 }
54
55 @Override
56 public void init(FileSystem fs, Path path, Configuration conf, FSDataInputStream stream)
57 throws IOException {
58 this.conf = conf;
59 this.path = path;
60 this.fs = fs;
61 this.fileLength = this.fs.getFileStatus(path).getLen();
62 this.trailerWarnSize = conf.getInt(HLog.WAL_TRAILER_WARN_SIZE,
63 HLog.DEFAULT_WAL_TRAILER_WARN_SIZE);
64 initReader(stream);
65
66 boolean compression = hasCompression();
67 if (compression) {
68
69 try {
70 if (compressionContext == null) {
71 compressionContext = new CompressionContext(LRUDictionary.class,
72 FSUtils.isRecoveredEdits(path));
73 } else {
74 compressionContext.clear();
75 }
76 } catch (Exception e) {
77 throw new IOException("Failed to initialize CompressionContext", e);
78 }
79 }
80 initAfterCompression();
81 }
82
83 @Override
84 public HLog.Entry next() throws IOException {
85 return next(null);
86 }
87
88 @Override
89 public HLog.Entry next(HLog.Entry reuse) throws IOException {
90 HLog.Entry e = reuse;
91 if (e == null) {
92 e = new HLog.Entry(new HLogKey(), new WALEdit());
93 }
94 if (compressionContext != null) {
95 e.setCompressionContext(compressionContext);
96 }
97
98 boolean hasEntry = readNext(e);
99 edit++;
100 if (compressionContext != null && emptyCompressionContext) {
101 emptyCompressionContext = false;
102 }
103 return hasEntry ? e : null;
104 }
105
106
107 @Override
108 public void seek(long pos) throws IOException {
109 if (compressionContext != null && emptyCompressionContext) {
110 while (next() != null) {
111 if (getPosition() == pos) {
112 emptyCompressionContext = false;
113 break;
114 }
115 }
116 }
117 seekOnFs(pos);
118 }
119
120
121
122
123
124 protected abstract void initReader(FSDataInputStream stream) throws IOException;
125
126
127
128
129 protected abstract void initAfterCompression() throws IOException;
130
131
132
133 protected abstract boolean hasCompression();
134
135
136
137
138
139
140 protected abstract boolean readNext(HLog.Entry e) throws IOException;
141
142
143
144
145 protected abstract void seekOnFs(long pos) throws IOException;
146
147 @Override
148 public WALTrailer getWALTrailer() {
149 return null;
150 }
151 }