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.Arrays;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.classification.InterfaceAudience;
31 import org.apache.hadoop.fs.FSDataInputStream;
32 import org.apache.hadoop.hbase.codec.Codec;
33 import org.apache.hadoop.hbase.io.LimitInputStream;
34 import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
35 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey;
36 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
37 import org.apache.hadoop.hbase.util.Bytes;
38
39 import com.google.protobuf.CodedInputStream;
40 import com.google.protobuf.InvalidProtocolBufferException;
41
42
43
44
45
46
47
48
49
50
51
52
53 @InterfaceAudience.Private
54 public class ProtobufLogReader extends ReaderBase {
55 private static final Log LOG = LogFactory.getLog(ProtobufLogReader.class);
56 static final byte[] PB_WAL_MAGIC = Bytes.toBytes("PWAL");
57 static final byte[] PB_WAL_COMPLETE_MAGIC = Bytes.toBytes("LAWP");
58 private FSDataInputStream inputStream;
59 private Codec.Decoder cellDecoder;
60 private WALCellCodec.ByteStringUncompressor byteStringUncompressor;
61 private boolean hasCompression = false;
62
63
64 private long walEditsStopOffset;
65 private boolean trailerPresent;
66
67 public ProtobufLogReader() {
68 super();
69 }
70
71 @Override
72 public void close() throws IOException {
73 if (this.inputStream != null) {
74 this.inputStream.close();
75 this.inputStream = null;
76 }
77 }
78
79 @Override
80 public long getPosition() throws IOException {
81 return inputStream.getPos();
82 }
83
84 @Override
85 public void reset() throws IOException {
86 initInternal(null, false);
87 initAfterCompression();
88 }
89
90 @Override
91 protected void initReader(FSDataInputStream stream) throws IOException {
92 initInternal(stream, true);
93 }
94
95 private void initInternal(FSDataInputStream stream, boolean isFirst) throws IOException {
96 close();
97 long expectedPos = PB_WAL_MAGIC.length;
98 if (stream == null) {
99 stream = fs.open(path);
100 stream.seek(expectedPos);
101 }
102 if (stream.getPos() != expectedPos) {
103 throw new IOException("The stream is at invalid position: " + stream.getPos());
104 }
105
106 WALProtos.WALHeader.Builder builder = WALProtos.WALHeader.newBuilder();
107 boolean hasHeader = builder.mergeDelimitedFrom(stream);
108 if (!hasHeader) {
109 throw new EOFException("Couldn't read WAL PB header");
110 }
111 if (isFirst) {
112 WALProtos.WALHeader header = builder.build();
113 this.hasCompression = header.hasHasCompression() && header.getHasCompression();
114 }
115 this.inputStream = stream;
116 this.walEditsStopOffset = this.fileLength;
117 long currentPosition = stream.getPos();
118 trailerPresent = setTrailerIfPresent();
119 this.seekOnFs(currentPosition);
120 if (LOG.isTraceEnabled()) {
121 LOG.trace("After reading the trailer: walEditsStopOffset: " + this.walEditsStopOffset
122 + ", fileLength: " + this.fileLength + ", " + "trailerPresent: " + trailerPresent);
123 }
124 }
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143 private boolean setTrailerIfPresent() {
144 try {
145 long trailerSizeOffset = this.fileLength - (PB_WAL_COMPLETE_MAGIC.length + Bytes.SIZEOF_INT);
146 if (trailerSizeOffset <= 0) return false;
147 this.seekOnFs(trailerSizeOffset);
148
149 int trailerSize = this.inputStream.readInt();
150 ByteBuffer buf = ByteBuffer.allocate(ProtobufLogReader.PB_WAL_COMPLETE_MAGIC.length);
151 this.inputStream.readFully(buf.array(), buf.arrayOffset(), buf.capacity());
152 if (!Arrays.equals(buf.array(), PB_WAL_COMPLETE_MAGIC)) {
153 LOG.trace("No trailer found.");
154 return false;
155 }
156 if (trailerSize < 0) {
157 LOG.warn("Invalid trailer Size " + trailerSize + ", ignoring the trailer");
158 return false;
159 } else if (trailerSize > this.trailerWarnSize) {
160
161 LOG.warn("Please investigate WALTrailer usage. Trailer size > maximum configured size : "
162 + trailerSize + " > " + this.trailerWarnSize);
163 }
164
165 long positionOfTrailer = trailerSizeOffset - trailerSize;
166 this.seekOnFs(positionOfTrailer);
167
168 buf = ByteBuffer.allocate(trailerSize);
169 this.inputStream.readFully(buf.array(), buf.arrayOffset(), buf.capacity());
170 trailer = WALTrailer.parseFrom(buf.array());
171 this.walEditsStopOffset = positionOfTrailer;
172 return true;
173 } catch (IOException ioe) {
174 LOG.warn("Got IOE while reading the trailer. Continuing as if no trailer is present.", ioe);
175 }
176 return false;
177 }
178
179 @Override
180 protected void initAfterCompression() throws IOException {
181 WALCellCodec codec = WALCellCodec.create(this.conf, this.compressionContext);
182 this.cellDecoder = codec.getDecoder(this.inputStream);
183 if (this.hasCompression) {
184 this.byteStringUncompressor = codec.getByteStringUncompressor();
185 }
186 }
187
188 @Override
189 protected boolean hasCompression() {
190 return this.hasCompression;
191 }
192
193 @Override
194 protected boolean readNext(HLog.Entry entry) throws IOException {
195 while (true) {
196
197 long originalPosition = this.inputStream.getPos();
198 if (trailerPresent && originalPosition > 0 && originalPosition == this.walEditsStopOffset) {
199 return false;
200 }
201 WALKey.Builder builder = WALKey.newBuilder();
202 long size = 0;
203 try {
204 long available = -1;
205 try {
206 int firstByte = this.inputStream.read();
207 if (firstByte == -1) {
208 throw new EOFException("First byte is negative");
209 }
210 size = CodedInputStream.readRawVarint32(firstByte, this.inputStream);
211
212 available = this.inputStream.available();
213 if (available > 0 && available < size) {
214 throw new EOFException("Available stream not enough for edit, " +
215 "inputStream.available()= " + this.inputStream.available() + ", " +
216 "entry size= " + size);
217 }
218 final InputStream limitedInput = new LimitInputStream(this.inputStream, size);
219 builder.mergeFrom(limitedInput);
220 } catch (InvalidProtocolBufferException ipbe) {
221 throw (EOFException) new EOFException("Invalid PB, EOF? Ignoring; originalPosition=" +
222 originalPosition + ", currentPosition=" + this.inputStream.getPos() +
223 ", messageSize=" + size + ", currentAvailable=" + available).initCause(ipbe);
224 }
225 if (!builder.isInitialized()) {
226
227
228 throw new EOFException("Partial PB while reading WAL, " +
229 "probably an unexpected EOF, ignoring");
230 }
231 WALKey walKey = builder.build();
232 entry.getKey().readFieldsFromPb(walKey, this.byteStringUncompressor);
233 if (!walKey.hasFollowingKvCount() || 0 == walKey.getFollowingKvCount()) {
234 LOG.trace("WALKey has no KVs that follow it; trying the next one");
235 continue;
236 }
237 int expectedCells = walKey.getFollowingKvCount();
238 long posBefore = this.inputStream.getPos();
239 try {
240 int actualCells = entry.getEdit().readFromCells(cellDecoder, expectedCells);
241 if (expectedCells != actualCells) {
242 throw new EOFException("Only read " + actualCells);
243 }
244 } catch (Exception ex) {
245 String posAfterStr = "<unknown>";
246 try {
247 posAfterStr = this.inputStream.getPos() + "";
248 } catch (Throwable t) {
249 LOG.trace("Error getting pos for error message - ignoring", t);
250 }
251 String message = " while reading " + expectedCells + " WAL KVs; started reading at "
252 + posBefore + " and read up to " + posAfterStr;
253 IOException realEofEx = extractHiddenEof(ex);
254 throw (EOFException) new EOFException("EOF " + message).
255 initCause(realEofEx != null ? ex : realEofEx);
256 }
257 if (trailerPresent && this.inputStream.getPos() > this.walEditsStopOffset) {
258 LOG.error("Read WALTrailer while reading WALEdits. hlog: " + this.path
259 + ", inputStream.getPos(): " + this.inputStream.getPos() + ", walEditsStopOffset: "
260 + this.walEditsStopOffset);
261 throw new EOFException("Read WALTrailer while reading WALEdits");
262 }
263 } catch (EOFException eof) {
264 LOG.trace("Encountered a malformed edit, seeking back to last good position in file", eof);
265
266 if (originalPosition < 0) throw eof;
267
268
269 seekOnFs(originalPosition);
270 return false;
271 }
272 return true;
273 }
274 }
275
276 private IOException extractHiddenEof(Exception ex) {
277
278
279 IOException ioEx = null;
280 if (ex instanceof EOFException) {
281 return (EOFException)ex;
282 } else if (ex instanceof IOException) {
283 ioEx = (IOException)ex;
284 } else if (ex instanceof RuntimeException
285 && ex.getCause() != null && ex.getCause() instanceof IOException) {
286 ioEx = (IOException)ex.getCause();
287 }
288 if (ioEx != null) {
289 if (ioEx.getMessage().contains("EOF")) return ioEx;
290 return null;
291 }
292 return null;
293 }
294
295 @Override
296 public WALTrailer getWALTrailer() {
297 return trailer;
298 }
299
300 @Override
301 protected void seekOnFs(long pos) throws IOException {
302 this.inputStream.seek(pos);
303 }
304 }