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