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.EOFException;
23 import java.io.IOException;
24 import java.io.InputStream;
25 import java.nio.ByteBuffer;
26 import java.util.ArrayList;
27 import java.util.Arrays;
28 import java.util.List;
29
30 import org.apache.commons.logging.Log;
31 import org.apache.commons.logging.LogFactory;
32 import org.apache.hadoop.hbase.classification.InterfaceAudience;
33 import org.apache.hadoop.conf.Configuration;
34 import org.apache.hadoop.fs.Path;
35 import org.apache.hadoop.fs.FileSystem;
36 import org.apache.hadoop.fs.FSDataInputStream;
37 import org.apache.hadoop.hbase.codec.Codec;
38 import org.apache.hadoop.hbase.HBaseInterfaceAudience;
39 import org.apache.hadoop.hbase.io.LimitInputStream;
40 import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
41 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALHeader.Builder;
42 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey;
43 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
44 import org.apache.hadoop.hbase.util.Bytes;
45 import org.apache.hadoop.hbase.wal.WAL.Entry;
46
47 import com.google.protobuf.CodedInputStream;
48 import com.google.protobuf.InvalidProtocolBufferException;
49
50
51
52
53
54
55
56
57
58
59
60
61 @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX, HBaseInterfaceAudience.CONFIG})
62 public class ProtobufLogReader extends ReaderBase {
63 private static final Log LOG = LogFactory.getLog(ProtobufLogReader.class);
64
65 @InterfaceAudience.Private
66 public static final byte[] PB_WAL_MAGIC = Bytes.toBytes("PWAL");
67
68 @InterfaceAudience.Private
69 public static final byte[] PB_WAL_COMPLETE_MAGIC = Bytes.toBytes("LAWP");
70
71
72
73
74 static final String WAL_TRAILER_WARN_SIZE = "hbase.regionserver.waltrailer.warn.size";
75 static final int DEFAULT_WAL_TRAILER_WARN_SIZE = 1024 * 1024;
76
77 protected FSDataInputStream inputStream;
78 protected Codec.Decoder cellDecoder;
79 protected WALCellCodec.ByteStringUncompressor byteStringUncompressor;
80 protected boolean hasCompression = false;
81 protected boolean hasTagCompression = false;
82
83
84 private long walEditsStopOffset;
85 private boolean trailerPresent;
86 protected WALTrailer trailer;
87
88
89 protected int trailerWarnSize;
90 private static List<String> writerClsNames = new ArrayList<String>();
91 static {
92 writerClsNames.add(ProtobufLogWriter.class.getSimpleName());
93 }
94
95 enum WALHdrResult {
96 EOF,
97 SUCCESS,
98 UNKNOWN_WRITER_CLS
99 }
100
101
102 static class WALHdrContext {
103 WALHdrResult result;
104 String cellCodecClsName;
105
106 WALHdrContext(WALHdrResult result, String cellCodecClsName) {
107 this.result = result;
108 this.cellCodecClsName = cellCodecClsName;
109 }
110 WALHdrResult getResult() {
111 return result;
112 }
113 String getCellCodecClsName() {
114 return cellCodecClsName;
115 }
116 }
117
118 public ProtobufLogReader() {
119 super();
120 }
121
122 @Override
123 public void close() throws IOException {
124 if (this.inputStream != null) {
125 this.inputStream.close();
126 this.inputStream = null;
127 }
128 }
129
130 @Override
131 public long getPosition() throws IOException {
132 return inputStream.getPos();
133 }
134
135 @Override
136 public void reset() throws IOException {
137 String clsName = initInternal(null, false);
138 initAfterCompression(clsName);
139 }
140
141 @Override
142 public void init(FileSystem fs, Path path, Configuration conf, FSDataInputStream stream)
143 throws IOException {
144 this.trailerWarnSize = conf.getInt(WAL_TRAILER_WARN_SIZE, DEFAULT_WAL_TRAILER_WARN_SIZE);
145 super.init(fs, path, conf, stream);
146 }
147
148 @Override
149 protected String initReader(FSDataInputStream stream) throws IOException {
150 return initInternal(stream, true);
151 }
152
153
154
155
156 protected List<String> getWriterClsNames() {
157 return writerClsNames;
158 }
159
160 protected WALHdrContext readHeader(Builder builder, FSDataInputStream stream)
161 throws IOException {
162 boolean res = builder.mergeDelimitedFrom(stream);
163 if (!res) return new WALHdrContext(WALHdrResult.EOF, null);
164 if (builder.hasWriterClsName() &&
165 !getWriterClsNames().contains(builder.getWriterClsName())) {
166 return new WALHdrContext(WALHdrResult.UNKNOWN_WRITER_CLS, null);
167 }
168 String clsName = null;
169 if (builder.hasCellCodecClsName()) {
170 clsName = builder.getCellCodecClsName();
171 }
172 return new WALHdrContext(WALHdrResult.SUCCESS, clsName);
173 }
174
175 private String initInternal(FSDataInputStream stream, boolean isFirst)
176 throws IOException {
177 close();
178 long expectedPos = PB_WAL_MAGIC.length;
179 if (stream == null) {
180 stream = fs.open(path);
181 stream.seek(expectedPos);
182 }
183 if (stream.getPos() != expectedPos) {
184 throw new IOException("The stream is at invalid position: " + stream.getPos());
185 }
186
187 WALProtos.WALHeader.Builder builder = WALProtos.WALHeader.newBuilder();
188 WALHdrContext hdrCtxt = readHeader(builder, stream);
189 WALHdrResult walHdrRes = hdrCtxt.getResult();
190 if (walHdrRes == WALHdrResult.EOF) {
191 throw new EOFException("Couldn't read WAL PB header");
192 }
193 if (walHdrRes == WALHdrResult.UNKNOWN_WRITER_CLS) {
194 throw new IOException("Got unknown writer class: " + builder.getWriterClsName());
195 }
196 if (isFirst) {
197 WALProtos.WALHeader header = builder.build();
198 this.hasCompression = header.hasHasCompression() && header.getHasCompression();
199 this.hasTagCompression = header.hasHasTagCompression() && header.getHasTagCompression();
200 }
201 this.inputStream = stream;
202 this.walEditsStopOffset = this.fileLength;
203 long currentPosition = stream.getPos();
204 trailerPresent = setTrailerIfPresent();
205 this.seekOnFs(currentPosition);
206 if (LOG.isTraceEnabled()) {
207 LOG.trace("After reading the trailer: walEditsStopOffset: " + this.walEditsStopOffset
208 + ", fileLength: " + this.fileLength + ", " + "trailerPresent: " + trailerPresent);
209 }
210 return hdrCtxt.getCellCodecClsName();
211 }
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230 private boolean setTrailerIfPresent() {
231 try {
232 long trailerSizeOffset = this.fileLength - (PB_WAL_COMPLETE_MAGIC.length + Bytes.SIZEOF_INT);
233 if (trailerSizeOffset <= 0) return false;
234 this.seekOnFs(trailerSizeOffset);
235
236 int trailerSize = this.inputStream.readInt();
237 ByteBuffer buf = ByteBuffer.allocate(ProtobufLogReader.PB_WAL_COMPLETE_MAGIC.length);
238 this.inputStream.readFully(buf.array(), buf.arrayOffset(), buf.capacity());
239 if (!Arrays.equals(buf.array(), PB_WAL_COMPLETE_MAGIC)) {
240 LOG.trace("No trailer found.");
241 return false;
242 }
243 if (trailerSize < 0) {
244 LOG.warn("Invalid trailer Size " + trailerSize + ", ignoring the trailer");
245 return false;
246 } else if (trailerSize > this.trailerWarnSize) {
247
248 LOG.warn("Please investigate WALTrailer usage. Trailer size > maximum configured size : "
249 + trailerSize + " > " + this.trailerWarnSize);
250 }
251
252 long positionOfTrailer = trailerSizeOffset - trailerSize;
253 this.seekOnFs(positionOfTrailer);
254
255 buf = ByteBuffer.allocate(trailerSize);
256 this.inputStream.readFully(buf.array(), buf.arrayOffset(), buf.capacity());
257 trailer = WALTrailer.parseFrom(buf.array());
258 this.walEditsStopOffset = positionOfTrailer;
259 return true;
260 } catch (IOException ioe) {
261 LOG.warn("Got IOE while reading the trailer. Continuing as if no trailer is present.", ioe);
262 }
263 return false;
264 }
265
266 protected WALCellCodec getCodec(Configuration conf, String cellCodecClsName,
267 CompressionContext compressionContext) throws IOException {
268 return WALCellCodec.create(conf, cellCodecClsName, compressionContext);
269 }
270
271 @Override
272 protected void initAfterCompression() throws IOException {
273 initAfterCompression(null);
274 }
275
276 @Override
277 protected void initAfterCompression(String cellCodecClsName) throws IOException {
278 WALCellCodec codec = getCodec(this.conf, cellCodecClsName, this.compressionContext);
279 this.cellDecoder = codec.getDecoder(this.inputStream);
280 if (this.hasCompression) {
281 this.byteStringUncompressor = codec.getByteStringUncompressor();
282 }
283 }
284
285 @Override
286 protected boolean hasCompression() {
287 return this.hasCompression;
288 }
289
290 @Override
291 protected boolean hasTagCompression() {
292 return this.hasTagCompression;
293 }
294
295 @Override
296 protected boolean readNext(Entry entry) throws IOException {
297 while (true) {
298
299 long originalPosition = this.inputStream.getPos();
300 if (trailerPresent && originalPosition > 0 && originalPosition == this.walEditsStopOffset) {
301 return false;
302 }
303 WALKey.Builder builder = WALKey.newBuilder();
304 long size = 0;
305 try {
306 long available = -1;
307 try {
308 int firstByte = this.inputStream.read();
309 if (firstByte == -1) {
310 throw new EOFException("First byte is negative");
311 }
312 size = CodedInputStream.readRawVarint32(firstByte, this.inputStream);
313
314 available = this.inputStream.available();
315 if (available > 0 && available < size) {
316 throw new EOFException("Available stream not enough for edit, " +
317 "inputStream.available()= " + this.inputStream.available() + ", " +
318 "entry size= " + size);
319 }
320 final InputStream limitedInput = new LimitInputStream(this.inputStream, size);
321 builder.mergeFrom(limitedInput);
322 } catch (InvalidProtocolBufferException ipbe) {
323 throw (EOFException) new EOFException("Invalid PB, EOF? Ignoring; originalPosition=" +
324 originalPosition + ", currentPosition=" + this.inputStream.getPos() +
325 ", messageSize=" + size + ", currentAvailable=" + available).initCause(ipbe);
326 }
327 if (!builder.isInitialized()) {
328
329
330 throw new EOFException("Partial PB while reading WAL, " +
331 "probably an unexpected EOF, ignoring");
332 }
333 WALKey walKey = builder.build();
334 entry.getKey().readFieldsFromPb(walKey, this.byteStringUncompressor);
335 if (!walKey.hasFollowingKvCount() || 0 == walKey.getFollowingKvCount()) {
336 LOG.trace("WALKey has no KVs that follow it; trying the next one");
337 continue;
338 }
339 int expectedCells = walKey.getFollowingKvCount();
340 long posBefore = this.inputStream.getPos();
341 try {
342 int actualCells = entry.getEdit().readFromCells(cellDecoder, expectedCells);
343 if (expectedCells != actualCells) {
344 throw new EOFException("Only read " + actualCells);
345 }
346 } catch (Exception ex) {
347 String posAfterStr = "<unknown>";
348 try {
349 posAfterStr = this.inputStream.getPos() + "";
350 } catch (Throwable t) {
351 LOG.trace("Error getting pos for error message - ignoring", t);
352 }
353 String message = " while reading " + expectedCells + " WAL KVs; started reading at "
354 + posBefore + " and read up to " + posAfterStr;
355 IOException realEofEx = extractHiddenEof(ex);
356 throw (EOFException) new EOFException("EOF " + message).
357 initCause(realEofEx != null ? realEofEx : ex);
358 }
359 if (trailerPresent && this.inputStream.getPos() > this.walEditsStopOffset) {
360 LOG.error("Read WALTrailer while reading WALEdits. wal: " + this.path
361 + ", inputStream.getPos(): " + this.inputStream.getPos() + ", walEditsStopOffset: "
362 + this.walEditsStopOffset);
363 throw new EOFException("Read WALTrailer while reading WALEdits");
364 }
365 } catch (EOFException eof) {
366 LOG.trace("Encountered a malformed edit, seeking back to last good position in file", eof);
367
368 if (originalPosition < 0) throw eof;
369
370
371 seekOnFs(originalPosition);
372 return false;
373 }
374 return true;
375 }
376 }
377
378 private IOException extractHiddenEof(Exception ex) {
379
380
381 IOException ioEx = null;
382 if (ex instanceof EOFException) {
383 return (EOFException)ex;
384 } else if (ex instanceof IOException) {
385 ioEx = (IOException)ex;
386 } else if (ex instanceof RuntimeException
387 && ex.getCause() != null && ex.getCause() instanceof IOException) {
388 ioEx = (IOException)ex.getCause();
389 }
390 if (ioEx != null) {
391 if (ioEx.getMessage().contains("EOF")) return ioEx;
392 return null;
393 }
394 return null;
395 }
396
397 @Override
398 protected void seekOnFs(long pos) throws IOException {
399 this.inputStream.seek(pos);
400 }
401 }