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