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 import java.util.NavigableMap;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.hbase.classification.InterfaceAudience;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.fs.FSDataInputStream;
33 import org.apache.hadoop.fs.FileSystem;
34 import org.apache.hadoop.fs.Path;
35 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
36 import org.apache.hadoop.hbase.wal.WAL.Entry;
37 import org.apache.hadoop.io.SequenceFile;
38 import org.apache.hadoop.io.SequenceFile.Metadata;
39 import org.apache.hadoop.io.Text;
40
41 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX, HBaseInterfaceAudience.CONFIG})
42 public class SequenceFileLogReader extends ReaderBase {
43 private static final Log LOG = LogFactory.getLog(SequenceFileLogReader.class);
44
45
46 private static final Text WAL_VERSION_KEY = new Text("version");
47
48
49
50 private static final int COMPRESSION_VERSION = 1;
51 private static final Text WAL_COMPRESSION_TYPE_KEY = new Text("compression.type");
52 private static final Text DICTIONARY_COMPRESSION_TYPE = new Text("dictionary");
53
54
55
56
57
58
59
60
61
62
63
64
65
66 private static class WALReader extends SequenceFile.Reader {
67
68 WALReader(final FileSystem fs, final Path p, final Configuration c)
69 throws IOException {
70 super(fs, p, c);
71 }
72
73 @Override
74 protected FSDataInputStream openFile(FileSystem fs, Path file,
75 int bufferSize, long length)
76 throws IOException {
77 return new WALReaderFSDataInputStream(super.openFile(fs, file,
78 bufferSize, length), length);
79 }
80
81
82
83
84 static class WALReaderFSDataInputStream extends FSDataInputStream {
85 private boolean firstGetPosInvocation = true;
86 private long length;
87
88 WALReaderFSDataInputStream(final FSDataInputStream is, final long l)
89 throws IOException {
90 super(is);
91 this.length = l;
92 }
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109 @Override
110 public long getPos() throws IOException {
111 if (this.firstGetPosInvocation) {
112 this.firstGetPosInvocation = false;
113 long adjust = 0;
114
115 try {
116 Field fIn = FilterInputStream.class.getDeclaredField("in");
117 fIn.setAccessible(true);
118 Object realIn = fIn.get(this.in);
119
120
121 if (realIn.getClass().getName().endsWith("DFSInputStream")) {
122 Method getFileLength = realIn.getClass().
123 getDeclaredMethod("getFileLength", new Class<?> []{});
124 getFileLength.setAccessible(true);
125 long realLength = ((Long)getFileLength.
126 invoke(realIn, new Object []{})).longValue();
127 assert(realLength >= this.length);
128 adjust = realLength - this.length;
129 } else {
130 LOG.info("Input stream class: " + realIn.getClass().getName() +
131 ", not adjusting length");
132 }
133 } catch(Exception e) {
134 SequenceFileLogReader.LOG.warn(
135 "Error while trying to get accurate file length. " +
136 "Truncation / data loss may occur if RegionServers die.", e);
137 }
138
139 return adjust + super.getPos();
140 }
141 return super.getPos();
142 }
143 }
144 }
145
146
147 protected SequenceFile.Reader reader;
148 long entryStart = 0;
149
150 public SequenceFileLogReader() {
151 super();
152 }
153
154 @Override
155 public void close() throws IOException {
156 try {
157 if (reader != null) {
158 this.reader.close();
159 this.reader = null;
160 }
161 } catch (IOException ioe) {
162 throw addFileInfoToException(ioe);
163 }
164 }
165
166 @Override
167 public long getPosition() throws IOException {
168 return reader != null ? reader.getPosition() : 0;
169 }
170
171 @Override
172 public void reset() throws IOException {
173
174
175 reader = new WALReader(fs, path, conf);
176 }
177
178 @Override
179 protected String initReader(FSDataInputStream stream) throws IOException {
180
181 if (stream != null) {
182 stream.close();
183 }
184 reset();
185 return null;
186 }
187
188 @Override
189 protected void initAfterCompression(String cellCodecClsName) throws IOException {
190
191 }
192
193 @Override
194 protected void initAfterCompression() throws IOException {
195
196 }
197
198 @Override
199 protected boolean hasCompression() {
200 return isWALCompressionEnabled(reader.getMetadata());
201 }
202
203 @Override
204 protected boolean hasTagCompression() {
205
206 return false;
207 }
208
209
210
211
212
213 static boolean isWALCompressionEnabled(final Metadata metadata) {
214
215 Text txt = metadata.get(WAL_VERSION_KEY);
216 if (txt == null || Integer.parseInt(txt.toString()) < COMPRESSION_VERSION) {
217 return false;
218 }
219
220 txt = metadata.get(WAL_COMPRESSION_TYPE_KEY);
221 return txt != null && txt.equals(DICTIONARY_COMPRESSION_TYPE);
222 }
223
224
225
226
227
228
229
230
231 @Override
232 protected boolean readNext(Entry e) throws IOException {
233 try {
234 if (!(e.getKey() instanceof HLogKey)) {
235 final IllegalArgumentException exception = new IllegalArgumentException(
236 "SequenceFileLogReader only works when given entries that have HLogKey for keys. This" +
237 " one had '" + e.getKey().getClass() + "'");
238 LOG.error("We need to use the legacy SequenceFileLogReader to handle a " +
239 " pre-0.96 style WAL, but HBase internals failed to use the deprecated HLogKey class." +
240 " This is a bug; please file an issue or email the developer mailing list. You will " +
241 "need the following exception details when seeking help from the HBase community.",
242 exception);
243 throw exception;
244 }
245 boolean hasNext = this.reader.next((HLogKey)e.getKey(), e.getEdit());
246 if (!hasNext) return false;
247
248 NavigableMap<byte[], Integer> scopes = e.getEdit().getAndRemoveScopes();
249 if (scopes != null) {
250 e.getKey().readOlderScopes(scopes);
251 }
252 return true;
253 } catch (IOException ioe) {
254 throw addFileInfoToException(ioe);
255 }
256 }
257
258 @Override
259 protected void seekOnFs(long pos) throws IOException {
260 try {
261 reader.seek(pos);
262 } catch (IOException ioe) {
263 throw addFileInfoToException(ioe);
264 }
265 }
266
267 protected IOException addFileInfoToException(final IOException ioe)
268 throws IOException {
269 long pos = -1;
270 try {
271 pos = getPosition();
272 } catch (IOException e) {
273 LOG.warn("Failed getting position to add to throw", e);
274 }
275
276
277 long end = Long.MAX_VALUE;
278 try {
279 Field fEnd = SequenceFile.Reader.class.getDeclaredField("end");
280 fEnd.setAccessible(true);
281 end = fEnd.getLong(this.reader);
282 } catch(NoSuchFieldException nfe) {
283
284 } catch(IllegalAccessException iae) {
285
286 } catch(Exception e) {
287
288 LOG.warn("Unexpected exception when accessing the end field", e);
289 }
290
291 String msg = (this.path == null? "": this.path.toString()) +
292 ", entryStart=" + entryStart + ", pos=" + pos +
293 ((end == Long.MAX_VALUE) ? "" : ", end=" + end) +
294 ", edit=" + this.edit;
295
296
297 try {
298 return (IOException) ioe.getClass()
299 .getConstructor(String.class)
300 .newInstance(msg)
301 .initCause(ioe);
302 } catch(NoSuchMethodException nfe) {
303
304 } catch(IllegalAccessException iae) {
305
306 } catch(Exception e) {
307
308 LOG.warn("Unexpected exception when accessing the end field", e);
309 }
310 return ioe;
311 }
312 }