1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21 package org.apache.hadoop.hbase.regionserver.wal;
22
23 import java.io.FilterInputStream;
24 import java.io.IOException;
25 import java.lang.reflect.Field;
26 import java.lang.reflect.Method;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.conf.Configuration;
31 import org.apache.hadoop.fs.FSDataInputStream;
32 import org.apache.hadoop.fs.FileSystem;
33 import org.apache.hadoop.fs.Path;
34 import org.apache.hadoop.io.SequenceFile;
35
36 public class SequenceFileLogReader implements HLog.Reader {
37 private static final Log LOG = LogFactory.getLog(SequenceFileLogReader.class);
38
39
40
41
42
43
44
45
46
47
48
49
50
51 static class WALReader extends SequenceFile.Reader {
52
53 WALReader(final FileSystem fs, final Path p, final Configuration c)
54 throws IOException {
55 super(fs, p, c);
56 }
57
58 @Override
59 protected FSDataInputStream openFile(FileSystem fs, Path file,
60 int bufferSize, long length)
61 throws IOException {
62 return new WALReaderFSDataInputStream(super.openFile(fs, file,
63 bufferSize, length), length);
64 }
65
66
67
68
69
70
71 public boolean isWALCompressionEnabled() {
72 return SequenceFileLogWriter.isWALCompressionEnabled(this.getMetadata());
73 }
74
75
76
77
78 static class WALReaderFSDataInputStream extends FSDataInputStream {
79 private boolean firstGetPosInvocation = true;
80 private long length;
81
82 WALReaderFSDataInputStream(final FSDataInputStream is, final long l)
83 throws IOException {
84 super(is);
85 this.length = l;
86 }
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103 @Override
104 public long getPos() throws IOException {
105 if (this.firstGetPosInvocation) {
106 this.firstGetPosInvocation = false;
107 long adjust = 0;
108
109 try {
110 Field fIn = FilterInputStream.class.getDeclaredField("in");
111 fIn.setAccessible(true);
112 Object realIn = fIn.get(this.in);
113
114
115 if (realIn.getClass().getName().endsWith("DFSInputStream")) {
116 Method getFileLength = realIn.getClass().
117 getDeclaredMethod("getFileLength", new Class<?> []{});
118 getFileLength.setAccessible(true);
119 long realLength = ((Long)getFileLength.
120 invoke(realIn, new Object []{})).longValue();
121 assert(realLength >= this.length);
122 adjust = realLength - this.length;
123 } else {
124 LOG.info("Input stream class: " + realIn.getClass().getName() +
125 ", not adjusting length");
126 }
127 } catch(Exception e) {
128 SequenceFileLogReader.LOG.warn(
129 "Error while trying to get accurate file length. " +
130 "Truncation / data loss may occur if RegionServers die.", e);
131 }
132
133 return adjust + super.getPos();
134 }
135 return super.getPos();
136 }
137 }
138 }
139
140 Configuration conf;
141 WALReader reader;
142 FileSystem fs;
143
144
145 Path path;
146 int edit = 0;
147 long entryStart = 0;
148 boolean emptyCompressionContext = true;
149
150
151
152 protected CompressionContext compressionContext = null;
153
154 protected Class<? extends HLogKey> keyClass;
155 private WALEditCodec codec;
156
157
158
159
160 public SequenceFileLogReader() {
161 }
162
163
164
165
166
167
168
169 public SequenceFileLogReader(Class<? extends HLogKey> keyClass) {
170 this.keyClass = keyClass;
171 }
172
173 @Override
174 public void init(FileSystem fs, Path path, Configuration conf)
175 throws IOException {
176 this.conf = conf;
177 this.path = path;
178 reader = new WALReader(fs, path, conf);
179
180 this.fs = fs;
181
182
183 boolean compression = reader.isWALCompressionEnabled();
184 if (compression) {
185 try {
186 if (compressionContext == null) {
187 compressionContext = new CompressionContext(LRUDictionary.class);
188 } else {
189 compressionContext.clear();
190 }
191 } catch (Exception e) {
192 throw new IOException("Failed to initialize CompressionContext", e);
193 }
194 }
195
196
197 this.codec = WALEditCodec.create(conf, compressionContext);
198 }
199
200 @Override
201 public void close() throws IOException {
202 try {
203 if (reader != null) {
204 this.reader.close();
205 this.reader = null;
206 }
207 } catch (IOException ioe) {
208 throw addFileInfoToException(ioe);
209 }
210 }
211
212 @Override
213 public HLog.Entry next() throws IOException {
214 return next(null);
215 }
216
217 @Override
218 public HLog.Entry next(HLog.Entry reuse) throws IOException {
219 this.entryStart = this.reader.getPosition();
220 HLog.Entry e = reuse;
221 if (e == null) {
222 HLogKey key;
223 if (keyClass == null) {
224 key = HLog.newKey(conf);
225 } else {
226 try {
227 key = keyClass.newInstance();
228 } catch (InstantiationException ie) {
229 throw new IOException(ie);
230 } catch (IllegalAccessException iae) {
231 throw new IOException(iae);
232 }
233 }
234
235 WALEdit val = new WALEdit();
236 e = new HLog.Entry(key, val);
237 }
238
239 boolean b = false;
240 try {
241 e.getEdit().setCodec(codec);
242 if (compressionContext != null) {
243 e.getKey().setCompressionContext(compressionContext);
244 }
245 b = this.reader.next(e.getKey(), e.getEdit());
246 } catch (IOException ioe) {
247 throw addFileInfoToException(ioe);
248 }
249 edit++;
250 if (compressionContext != null && emptyCompressionContext) {
251 emptyCompressionContext = false;
252 }
253 return b? e: null;
254 }
255
256 @Override
257 public void seek(long pos) throws IOException {
258 if (compressionContext != null && emptyCompressionContext) {
259 while (next() != null) {
260 if (getPosition() == pos) {
261 emptyCompressionContext = false;
262 break;
263 }
264 }
265 }
266 try {
267 reader.seek(pos);
268 } catch (IOException ioe) {
269 throw addFileInfoToException(ioe);
270 }
271 }
272
273 @Override
274 public long getPosition() throws IOException {
275 return reader != null ? reader.getPosition() : 0;
276 }
277
278 protected IOException addFileInfoToException(final IOException ioe)
279 throws IOException {
280 long pos = -1;
281 try {
282 pos = getPosition();
283 } catch (IOException e) {
284 LOG.warn("Failed getting position to add to throw", e);
285 }
286
287
288 long end = Long.MAX_VALUE;
289 try {
290 Field fEnd = SequenceFile.Reader.class.getDeclaredField("end");
291 fEnd.setAccessible(true);
292 end = fEnd.getLong(this.reader);
293 } catch(Exception e) {
294
295 String msg = (this.path == null? "": this.path.toString()) +
296 ", entryStart=" + entryStart + ", pos=" + pos +
297 ((end == Long.MAX_VALUE) ? "" : ", end=" + end) +
298 ", edit=" + this.edit;
299
300
301 try {
302 return (IOException) ioe.getClass()
303 .getConstructor(String.class)
304 .newInstance(msg)
305 .initCause(ioe);
306 } catch(Exception e) {
307
308 return ioe;
309 }
310
311 @Override
312 public void reset() throws IOException {
313
314
315 reader = new WALReader(fs, path, conf);
316 }
317 }