View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
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.ArrayList;
26  import java.util.Arrays;
27  import java.util.List;
28  
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.hbase.classification.InterfaceAudience;
32  import org.apache.hadoop.conf.Configuration;
33  import org.apache.hadoop.fs.Path;
34  import org.apache.hadoop.fs.FileSystem;
35  import org.apache.hadoop.fs.FSDataInputStream;
36  import org.apache.hadoop.hbase.codec.Codec;
37  import org.apache.hadoop.hbase.io.LimitInputStream;
38  import org.apache.hadoop.hbase.HBaseInterfaceAudience;
39  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
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   * A Protobuf based WAL has the following structure:
52   * <p>
53   * &lt;PB_WAL_MAGIC&gt;&lt;WALHeader&gt;&lt;WALEdits&gt;...&lt;WALEdits&gt;&lt;Trailer&gt;
54   * &lt;TrailerSize&gt; &lt;PB_WAL_COMPLETE_MAGIC&gt;
55   * </p>
56   * The Reader reads meta information (WAL Compression state, WALTrailer, etc) in
57   * ProtobufLogReader#initReader(FSDataInputStream). A WALTrailer is an extensible structure
58   * which is appended at the end of the WAL. This is empty for now; it can contain some meta
59   * information such as Region level stats, etc in future.
60   */
61  @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX,
62    HBaseInterfaceAudience.CONFIG})
63  public class ProtobufLogReader extends ReaderBase {
64    private static final Log LOG = LogFactory.getLog(ProtobufLogReader.class);
65    // public for WALFactory until we move everything to o.a.h.h.wal
66    @InterfaceAudience.Private
67    public static final byte[] PB_WAL_MAGIC = Bytes.toBytes("PWAL");
68    // public for TestWALSplit
69    @InterfaceAudience.Private
70    public static final byte[] PB_WAL_COMPLETE_MAGIC = Bytes.toBytes("LAWP");
71    /**
72     * Configuration name of WAL Trailer's warning size. If a waltrailer's size is greater than the
73     * configured size, providers should log a warning. e.g. this is used with Protobuf reader/writer.
74     */
75    static final String WAL_TRAILER_WARN_SIZE = "hbase.regionserver.waltrailer.warn.size";
76    static final int DEFAULT_WAL_TRAILER_WARN_SIZE = 1024 * 1024; // 1MB
77  
78    protected FSDataInputStream inputStream;
79    protected Codec.Decoder cellDecoder;
80    protected WALCellCodec.ByteStringUncompressor byteStringUncompressor;
81    protected boolean hasCompression = false;
82    protected boolean hasTagCompression = false;
83    // walEditsStopOffset is the position of the last byte to read. After reading the last WALEdit
84    // entry in the wal, the inputstream's position is equal to walEditsStopOffset.
85    private long walEditsStopOffset;
86    private boolean trailerPresent;
87    protected WALTrailer trailer;
88    // maximum size of the wal Trailer in bytes. If a user writes/reads a trailer with size larger
89    // than this size, it is written/read respectively, with a WARN message in the log.
90    protected int trailerWarnSize;
91    private static List<String> writerClsNames = new ArrayList<String>();
92    static {
93      writerClsNames.add(ProtobufLogWriter.class.getSimpleName());
94    }
95    
96    // cell codec classname
97    private String codecClsName = null;
98  
99    @InterfaceAudience.Private
100   public long trailerSize() {
101     if (trailerPresent) {
102       // sizeof PB_WAL_COMPLETE_MAGIC + sizof trailerSize + trailer
103       final long calculatedSize = (long) PB_WAL_COMPLETE_MAGIC.length + Bytes.SIZEOF_INT +
104         trailer.getSerializedSize();
105       final long expectedSize = fileLength - walEditsStopOffset;
106       if (expectedSize != calculatedSize) {
107         LOG.warn("After parsing the trailer, we expect the total footer to be "+ expectedSize
108           + " bytes, but we calculate it as being " + calculatedSize);
109       }
110       return expectedSize;
111     } else {
112       return -1L;
113     }
114   }
115 
116   enum WALHdrResult {
117     EOF,                   // stream is at EOF when method starts
118     SUCCESS,
119     UNKNOWN_WRITER_CLS     // name of writer class isn't recognized
120   }
121   
122   // context for WALHdr carrying information such as Cell Codec classname
123   static class WALHdrContext {
124     WALHdrResult result;
125     String cellCodecClsName;
126     
127     WALHdrContext(WALHdrResult result, String cellCodecClsName) {
128       this.result = result;
129       this.cellCodecClsName = cellCodecClsName;
130     }
131     WALHdrResult getResult() {
132       return result;
133     }
134     String getCellCodecClsName() {
135       return cellCodecClsName;
136     }
137   }
138 
139   public ProtobufLogReader() {
140     super();
141   }
142 
143   @Override
144   public void close() throws IOException {
145     if (this.inputStream != null) {
146       this.inputStream.close();
147       this.inputStream = null;
148     }
149   }
150 
151   @Override
152   public long getPosition() throws IOException {
153     return inputStream.getPos();
154   }
155 
156   @Override
157   public void reset() throws IOException {
158     String clsName = initInternal(null, false);
159     initAfterCompression(clsName); // We need a new decoder (at least).
160   }
161 
162   @Override
163   public void init(FileSystem fs, Path path, Configuration conf, FSDataInputStream stream)
164       throws IOException {
165     this.trailerWarnSize = conf.getInt(WAL_TRAILER_WARN_SIZE, DEFAULT_WAL_TRAILER_WARN_SIZE);
166     super.init(fs, path, conf, stream);
167   }
168 
169   @Override
170   protected String initReader(FSDataInputStream stream) throws IOException {
171     return initInternal(stream, true);
172   }
173 
174   /*
175    * Returns names of the accepted writer classes
176    */
177   public List<String> getWriterClsNames() {
178     return writerClsNames;
179   }
180   
181   /*
182    * Returns the cell codec classname
183    */
184   public String getCodecClsName() {
185       return codecClsName;
186   }
187 
188   protected WALHdrContext readHeader(Builder builder, FSDataInputStream stream)
189       throws IOException {
190      boolean res = builder.mergeDelimitedFrom(stream);
191      if (!res) return new WALHdrContext(WALHdrResult.EOF, null);
192      if (builder.hasWriterClsName() &&
193          !getWriterClsNames().contains(builder.getWriterClsName())) {
194        return new WALHdrContext(WALHdrResult.UNKNOWN_WRITER_CLS, null);
195      }
196      String clsName = null;
197      if (builder.hasCellCodecClsName()) {
198        clsName = builder.getCellCodecClsName();
199      }
200      return new WALHdrContext(WALHdrResult.SUCCESS, clsName);
201   }
202 
203   private String initInternal(FSDataInputStream stream, boolean isFirst)
204       throws IOException {
205     close();
206     if (!isFirst) {
207       // Re-compute the file length.
208       this.fileLength = fs.getFileStatus(path).getLen();
209     }
210     long expectedPos = PB_WAL_MAGIC.length;
211     if (stream == null) {
212       stream = fs.open(path);
213       stream.seek(expectedPos);
214     }
215     if (stream.getPos() != expectedPos) {
216       throw new IOException("The stream is at invalid position: " + stream.getPos());
217     }
218     // Initialize metadata or, when we reset, just skip the header.
219     WALProtos.WALHeader.Builder builder = WALProtos.WALHeader.newBuilder();
220     WALHdrContext hdrCtxt = readHeader(builder, stream);
221     WALHdrResult walHdrRes = hdrCtxt.getResult();
222     if (walHdrRes == WALHdrResult.EOF) {
223       throw new EOFException("Couldn't read WAL PB header");
224     }
225     if (walHdrRes == WALHdrResult.UNKNOWN_WRITER_CLS) {
226       throw new IOException("Got unknown writer class: " + builder.getWriterClsName());
227     }
228     if (isFirst) {
229       WALProtos.WALHeader header = builder.build();
230       this.hasCompression = header.hasHasCompression() && header.getHasCompression();
231       this.hasTagCompression = header.hasHasTagCompression() && header.getHasTagCompression();
232     }
233     this.inputStream = stream;
234     this.walEditsStopOffset = this.fileLength;
235     long currentPosition = stream.getPos();
236     trailerPresent = setTrailerIfPresent();
237     this.seekOnFs(currentPosition);
238     if (LOG.isTraceEnabled()) {
239       LOG.trace("After reading the trailer: walEditsStopOffset: " + this.walEditsStopOffset
240           + ", fileLength: " + this.fileLength + ", " + "trailerPresent: " + (trailerPresent ? "true, size: " + trailer.getSerializedSize() : "false") + ", currentPosition: " + currentPosition);
241     }
242     
243     codecClsName = hdrCtxt.getCellCodecClsName();
244     
245     return hdrCtxt.getCellCodecClsName();
246   }
247 
248   /**
249    * To check whether a trailer is present in a WAL, it seeks to position (fileLength -
250    * PB_WAL_COMPLETE_MAGIC.size() - Bytes.SIZEOF_INT). It reads the int value to know the size of
251    * the trailer, and checks whether the trailer is present at the end or not by comparing the last
252    * PB_WAL_COMPLETE_MAGIC.size() bytes. In case trailer is not present, it returns false;
253    * otherwise, sets the trailer and sets this.walEditsStopOffset variable up to the point just
254    * before the trailer.
255    * <ul>
256    * The trailer is ignored in case:
257    * <li>fileLength is 0 or not correct (when file is under recovery, etc).
258    * <li>the trailer size is negative.
259    * </ul>
260    * <p>
261    * In case the trailer size > this.trailerMaxSize, it is read after a WARN message.
262    * @return true if a valid trailer is present
263    * @throws IOException
264    */
265   private boolean setTrailerIfPresent() {
266     try {
267       long trailerSizeOffset = this.fileLength - (PB_WAL_COMPLETE_MAGIC.length + Bytes.SIZEOF_INT);
268       if (trailerSizeOffset <= 0) return false;// no trailer possible.
269       this.seekOnFs(trailerSizeOffset);
270       // read the int as trailer size.
271       int trailerSize = this.inputStream.readInt();
272       ByteBuffer buf = ByteBuffer.allocate(ProtobufLogReader.PB_WAL_COMPLETE_MAGIC.length);
273       this.inputStream.readFully(buf.array(), buf.arrayOffset(), buf.capacity());
274       if (!Arrays.equals(buf.array(), PB_WAL_COMPLETE_MAGIC)) {
275         LOG.trace("No trailer found.");
276         return false;
277       }
278       if (trailerSize < 0) {
279         LOG.warn("Invalid trailer Size " + trailerSize + ", ignoring the trailer");
280         return false;
281       } else if (trailerSize > this.trailerWarnSize) {
282         // continue reading after warning the user.
283         LOG.warn("Please investigate WALTrailer usage. Trailer size > maximum configured size : "
284           + trailerSize + " > " + this.trailerWarnSize);
285       }
286       // seek to the position where trailer starts.
287       long positionOfTrailer = trailerSizeOffset - trailerSize;
288       this.seekOnFs(positionOfTrailer);
289       // read the trailer.
290       buf = ByteBuffer.allocate(trailerSize);// for trailer.
291       this.inputStream.readFully(buf.array(), buf.arrayOffset(), buf.capacity());
292       trailer = WALTrailer.parseFrom(buf.array());
293       this.walEditsStopOffset = positionOfTrailer;
294       return true;
295     } catch (IOException ioe) {
296       LOG.warn("Got IOE while reading the trailer. Continuing as if no trailer is present.", ioe);
297     }
298     return false;
299   }
300 
301   protected WALCellCodec getCodec(Configuration conf, String cellCodecClsName,
302       CompressionContext compressionContext) throws IOException {
303     return WALCellCodec.create(conf, cellCodecClsName, compressionContext);
304   }
305 
306   @Override
307   protected void initAfterCompression() throws IOException {
308     initAfterCompression(null);
309   }
310   
311   @Override
312   protected void initAfterCompression(String cellCodecClsName) throws IOException {
313     WALCellCodec codec = getCodec(this.conf, cellCodecClsName, this.compressionContext);
314     this.cellDecoder = codec.getDecoder(this.inputStream);
315     if (this.hasCompression) {
316       this.byteStringUncompressor = codec.getByteStringUncompressor();
317     }
318   }
319 
320   @Override
321   protected boolean hasCompression() {
322     return this.hasCompression;
323   }
324 
325   @Override
326   protected boolean hasTagCompression() {
327     return this.hasTagCompression;
328   }
329 
330   @Override
331   protected boolean readNext(Entry entry) throws IOException {
332     // OriginalPosition might be < 0 on local fs; if so, it is useless to us.
333     long originalPosition = this.inputStream.getPos();
334     if (trailerPresent && originalPosition > 0 && originalPosition == this.walEditsStopOffset) {
335       if (LOG.isTraceEnabled()) {
336         LOG.trace("Reached end of expected edits area at offset " + originalPosition);
337       }
338       return false;
339     }
340     WALKey.Builder builder = WALKey.newBuilder();
341     long size = 0;
342     boolean resetPosition = false;
343     try {
344       long available = -1;
345       try {
346         int firstByte = this.inputStream.read();
347         if (firstByte == -1) {
348           throw new EOFException("First byte is negative at offset " + originalPosition);
349         }
350         size = CodedInputStream.readRawVarint32(firstByte, this.inputStream);
351         // available may be < 0 on local fs for instance.  If so, can't depend on it.
352         available = this.inputStream.available();
353         if (available > 0 && available < size) {
354           throw new EOFException(
355             "Available stream not enough for edit, " + "inputStream.available()= "
356               + this.inputStream.available() + ", " + "entry size= " + size + " at offset = "
357               + this.inputStream.getPos());
358         }
359         ProtobufUtil.mergeFrom(builder, new LimitInputStream(this.inputStream, size), (int) size);
360       } catch (InvalidProtocolBufferException ipbe) {
361         resetPosition = true;
362         throw (EOFException) new EOFException(
363           "Invalid PB, EOF? Ignoring; originalPosition=" + originalPosition + ", currentPosition="
364             + this.inputStream.getPos() + ", messageSize=" + size + ", currentAvailable="
365             + available).initCause(ipbe);
366       }
367       if (!builder.isInitialized()) {
368         // TODO: not clear if we should try to recover from corrupt PB that looks semi-legit.
369         //       If we can get the KV count, we could, theoretically, try to get next record.
370         throw new EOFException(
371           "Partial PB while reading WAL, " + "probably an unexpected EOF, ignoring. current offset="
372             + this.inputStream.getPos());
373       }
374       WALKey walKey = builder.build();
375       entry.getKey().readFieldsFromPb(walKey, this.byteStringUncompressor);
376       if (!walKey.hasFollowingKvCount() || 0 == walKey.getFollowingKvCount()) {
377         if (LOG.isTraceEnabled()) {
378           LOG.trace("WALKey has no KVs that follow it; trying the next one. current offset="
379             + this.inputStream.getPos());
380         }
381         seekOnFs(originalPosition);
382         return false;
383       }
384       int expectedCells = walKey.getFollowingKvCount();
385       long posBefore = this.inputStream.getPos();
386       try {
387         int actualCells = entry.getEdit().readFromCells(cellDecoder, expectedCells);
388         if (expectedCells != actualCells) {
389           resetPosition = true;
390           throw new EOFException("Only read " + actualCells); // other info added in catch
391         }
392       } catch (Exception ex) {
393         String posAfterStr = "<unknown>";
394         try {
395           posAfterStr = this.inputStream.getPos() + "";
396         } catch (Throwable t) {
397           if (LOG.isTraceEnabled()) {
398             LOG.trace("Error getting pos for error message - ignoring", t);
399           }
400         }
401         String message =
402           " while reading " + expectedCells + " WAL KVs; started reading at " + posBefore
403             + " and read up to " + posAfterStr;
404         IOException realEofEx = extractHiddenEof(ex);
405         throw (EOFException) new EOFException("EOF " + message).
406           initCause(realEofEx != null ? realEofEx : ex);
407       }
408       if (trailerPresent && this.inputStream.getPos() > this.walEditsStopOffset) {
409         LOG.error(
410           "Read WALTrailer while reading WALEdits. wal: " + this.path + ", inputStream.getPos(): "
411             + this.inputStream.getPos() + ", walEditsStopOffset: " + this.walEditsStopOffset);
412         throw new EOFException("Read WALTrailer while reading WALEdits");
413       }
414     } catch (EOFException eof) {
415       // If originalPosition is < 0, it is rubbish and we cannot use it (probably local fs)
416       if (originalPosition < 0) {
417         if (LOG.isTraceEnabled()) {
418           LOG.trace("Encountered a malformed edit, but can't seek back to last good position "
419               + "because originalPosition is negative. last offset=" + this.inputStream.getPos(),
420             eof);
421         }
422         throw eof;
423       }
424       // If stuck at the same place and we got and exception, lets go back at the beginning.
425       if (inputStream.getPos() == originalPosition && resetPosition) {
426         if (LOG.isTraceEnabled()) {
427           LOG.trace("Encountered a malformed edit, seeking to the beginning of the WAL since "
428             + "current position and original position match at " + originalPosition);
429         }
430         seekOnFs(0);
431       } else {
432         // Else restore our position to original location in hope that next time through we will
433         // read successfully.
434         if (LOG.isTraceEnabled()) {
435           LOG.trace(
436             "Encountered a malformed edit, seeking back to last good position in file, " + "from "
437               + inputStream.getPos() + " to " + originalPosition, eof);
438         }
439         seekOnFs(originalPosition);
440       }
441       return false;
442     }
443     return true;
444   }
445 
446   private IOException extractHiddenEof(Exception ex) {
447     // There are two problems we are dealing with here. Hadoop stream throws generic exception
448     // for EOF, not EOFException; and scanner further hides it inside RuntimeException.
449     IOException ioEx = null;
450     if (ex instanceof EOFException) {
451       return (EOFException)ex;
452     } else if (ex instanceof IOException) {
453       ioEx = (IOException)ex;
454     } else if (ex instanceof RuntimeException
455         && ex.getCause() != null && ex.getCause() instanceof IOException) {
456       ioEx = (IOException)ex.getCause();
457     }
458     if (ioEx != null) {
459       if (ioEx.getMessage().contains("EOF")) return ioEx;
460       return null;
461     }
462     return null;
463   }
464 
465   @Override
466   protected void seekOnFs(long pos) throws IOException {
467     this.inputStream.seek(pos);
468   }
469 }