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.FSDataInputStream;
34  import org.apache.hadoop.hbase.codec.Codec;
35  import org.apache.hadoop.hbase.io.LimitInputStream;
36  import org.apache.hadoop.hbase.HBaseInterfaceAudience;
37  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
38  import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
39  import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALHeader.Builder;
40  import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALKey;
41  import org.apache.hadoop.hbase.protobuf.generated.WALProtos.WALTrailer;
42  import org.apache.hadoop.hbase.util.Bytes;
43  
44  import com.google.protobuf.CodedInputStream;
45  
46  /**
47   * A Protobuf based WAL has the following structure:
48   * <p>
49   * &lt;PB_WAL_MAGIC&gt;&lt;WALHeader&gt;&lt;WALEdits&gt;...&lt;WALEdits&gt;&lt;Trailer&gt;
50   * &lt;TrailerSize&gt; &lt;PB_WAL_COMPLETE_MAGIC&gt;
51   * </p>
52   * The Reader reads meta information (WAL Compression state, WALTrailer, etc) in
53   * {@link ProtobufLogReader#initReader(FSDataInputStream)}. A WALTrailer is an extensible structure
54   * which is appended at the end of the WAL. This is empty for now; it can contain some meta
55   * information such as Region level stats, etc in future.
56   */
57  @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX, HBaseInterfaceAudience.CONFIG})
58  public class ProtobufLogReader extends ReaderBase {
59    private static final Log LOG = LogFactory.getLog(ProtobufLogReader.class);
60    static final byte[] PB_WAL_MAGIC = Bytes.toBytes("PWAL");
61    static final byte[] PB_WAL_COMPLETE_MAGIC = Bytes.toBytes("LAWP");
62    protected FSDataInputStream inputStream;
63    protected Codec.Decoder cellDecoder;
64    protected WALCellCodec.ByteStringUncompressor byteStringUncompressor;
65    protected boolean hasCompression = false;
66    protected boolean hasTagCompression = false;
67    // walEditsStopOffset is the position of the last byte to read. After reading the last WALEdit entry
68    // in the hlog, the inputstream's position is equal to walEditsStopOffset.
69    private long walEditsStopOffset;
70    private boolean trailerPresent;
71    private static List<String> writerClsNames = new ArrayList<String>();
72    static {
73      writerClsNames.add(ProtobufLogWriter.class.getSimpleName());
74    }
75  
76    enum WALHdrResult {
77      EOF,                   // stream is at EOF when method starts
78      SUCCESS,
79      UNKNOWN_WRITER_CLS     // name of writer class isn't recognized
80    }
81    
82    // context for WALHdr carrying information such as Cell Codec classname
83    static class WALHdrContext {
84      WALHdrResult result;
85      String cellCodecClsName;
86      
87      WALHdrContext(WALHdrResult result, String cellCodecClsName) {
88        this.result = result;
89        this.cellCodecClsName = cellCodecClsName;
90      }
91      WALHdrResult getResult() {
92        return result;
93      }
94      String getCellCodecClsName() {
95        return cellCodecClsName;
96      }
97    }
98  
99    public ProtobufLogReader() {
100     super();
101   }
102 
103   @Override
104   public void close() throws IOException {
105     if (this.inputStream != null) {
106       this.inputStream.close();
107       this.inputStream = null;
108     }
109   }
110 
111   @Override
112   public long getPosition() throws IOException {
113     return inputStream.getPos();
114   }
115 
116   @Override
117   public void reset() throws IOException {
118     String clsName = initInternal(null, false);
119     initAfterCompression(clsName); // We need a new decoder (at least).
120   }
121 
122   @Override
123   protected String initReader(FSDataInputStream stream) throws IOException {
124     return initInternal(stream, true);
125   }
126 
127   /*
128    * Returns names of the accepted writer classes
129    */
130   protected List<String> getWriterClsNames() {
131     return writerClsNames;
132   }
133 
134   protected WALHdrContext readHeader(Builder builder, FSDataInputStream stream)
135       throws IOException {
136      boolean res = builder.mergeDelimitedFrom(stream);
137      if (!res) return new WALHdrContext(WALHdrResult.EOF, null);
138      if (builder.hasWriterClsName() &&
139          !getWriterClsNames().contains(builder.getWriterClsName())) {
140        return new WALHdrContext(WALHdrResult.UNKNOWN_WRITER_CLS, null);
141      }
142      String clsName = null;
143      if (builder.hasCellCodecClsName()) {
144        clsName = builder.getCellCodecClsName();
145      }
146      return new WALHdrContext(WALHdrResult.SUCCESS, clsName);
147   }
148 
149   private String initInternal(FSDataInputStream stream, boolean isFirst)
150       throws IOException {
151     close();
152     long expectedPos = PB_WAL_MAGIC.length;
153     if (stream == null) {
154       stream = fs.open(path);
155       stream.seek(expectedPos);
156     }
157     if (stream.getPos() != expectedPos) {
158       throw new IOException("The stream is at invalid position: " + stream.getPos());
159     }
160     // Initialize metadata or, when we reset, just skip the header.
161     WALProtos.WALHeader.Builder builder = WALProtos.WALHeader.newBuilder();
162     WALHdrContext hdrCtxt = readHeader(builder, stream);
163     WALHdrResult walHdrRes = hdrCtxt.getResult();
164     if (walHdrRes == WALHdrResult.EOF) {
165       throw new EOFException("Couldn't read WAL PB header");
166     }
167     if (walHdrRes == WALHdrResult.UNKNOWN_WRITER_CLS) {
168       throw new IOException("Got unknown writer class: " + builder.getWriterClsName());
169     }
170     if (isFirst) {
171       WALProtos.WALHeader header = builder.build();
172       this.hasCompression = header.hasHasCompression() && header.getHasCompression();
173       this.hasTagCompression = header.hasHasTagCompression() && header.getHasTagCompression();
174     }
175     this.inputStream = stream;
176     this.walEditsStopOffset = this.fileLength;
177     long currentPosition = stream.getPos();
178     trailerPresent = setTrailerIfPresent();
179     this.seekOnFs(currentPosition);
180     if (LOG.isTraceEnabled()) {
181       LOG.trace("After reading the trailer: walEditsStopOffset: " + this.walEditsStopOffset
182           + ", fileLength: " + this.fileLength + ", " + "trailerPresent: " + trailerPresent);
183     }
184     return hdrCtxt.getCellCodecClsName();
185   }
186 
187   /**
188    * To check whether a trailer is present in a WAL, it seeks to position (fileLength -
189    * PB_WAL_COMPLETE_MAGIC.size() - Bytes.SIZEOF_INT). It reads the int value to know the size of
190    * the trailer, and checks whether the trailer is present at the end or not by comparing the last
191    * PB_WAL_COMPLETE_MAGIC.size() bytes. In case trailer is not present, it returns false;
192    * otherwise, sets the trailer and sets this.walEditsStopOffset variable up to the point just
193    * before the trailer.
194    * <ul>
195    * The trailer is ignored in case:
196    * <li>fileLength is 0 or not correct (when file is under recovery, etc).
197    * <li>the trailer size is negative.
198    * </ul>
199    * <p>
200    * In case the trailer size > this.trailerMaxSize, it is read after a WARN message.
201    * @return true if a valid trailer is present
202    * @throws IOException
203    */
204   private boolean setTrailerIfPresent() {
205     try {
206       long trailerSizeOffset = this.fileLength - (PB_WAL_COMPLETE_MAGIC.length + Bytes.SIZEOF_INT);
207       if (trailerSizeOffset <= 0) return false;// no trailer possible.
208       this.seekOnFs(trailerSizeOffset);
209       // read the int as trailer size.
210       int trailerSize = this.inputStream.readInt();
211       ByteBuffer buf = ByteBuffer.allocate(ProtobufLogReader.PB_WAL_COMPLETE_MAGIC.length);
212       this.inputStream.readFully(buf.array(), buf.arrayOffset(), buf.capacity());
213       if (!Arrays.equals(buf.array(), PB_WAL_COMPLETE_MAGIC)) {
214         LOG.trace("No trailer found.");
215         return false;
216       }
217       if (trailerSize < 0) {
218         LOG.warn("Invalid trailer Size " + trailerSize + ", ignoring the trailer");
219         return false;
220       } else if (trailerSize > this.trailerWarnSize) {
221         // continue reading after warning the user.
222         LOG.warn("Please investigate WALTrailer usage. Trailer size > maximum configured size : "
223           + trailerSize + " > " + this.trailerWarnSize);
224       }
225       // seek to the position where trailer starts.
226       long positionOfTrailer = trailerSizeOffset - trailerSize;
227       this.seekOnFs(positionOfTrailer);
228       // read the trailer.
229       buf = ByteBuffer.allocate(trailerSize);// for trailer.
230       this.inputStream.readFully(buf.array(), buf.arrayOffset(), buf.capacity());
231       trailer = WALTrailer.parseFrom(buf.array());
232       this.walEditsStopOffset = positionOfTrailer;
233       return true;
234     } catch (IOException ioe) {
235       LOG.warn("Got IOE while reading the trailer. Continuing as if no trailer is present.", ioe);
236     }
237     return false;
238   }
239 
240   protected WALCellCodec getCodec(Configuration conf, String cellCodecClsName,
241       CompressionContext compressionContext) throws IOException {
242     return WALCellCodec.create(conf, cellCodecClsName, compressionContext);
243   }
244 
245   @Override
246   protected void initAfterCompression() throws IOException {
247     initAfterCompression(null);
248   }
249   
250   @Override
251   protected void initAfterCompression(String cellCodecClsName) throws IOException {
252     WALCellCodec codec = getCodec(this.conf, cellCodecClsName, this.compressionContext);
253     this.cellDecoder = codec.getDecoder(this.inputStream);
254     if (this.hasCompression) {
255       this.byteStringUncompressor = codec.getByteStringUncompressor();
256     }
257   }
258 
259   @Override
260   protected boolean hasCompression() {
261     return this.hasCompression;
262   }
263 
264   @Override
265   protected boolean hasTagCompression() {
266     return this.hasTagCompression;
267   }
268 
269   @Override
270   protected boolean readNext(HLog.Entry entry) throws IOException {
271     while (true) {
272       // OriginalPosition might be < 0 on local fs; if so, it is useless to us.
273       long originalPosition = this.inputStream.getPos();
274       if (trailerPresent && originalPosition > 0 && originalPosition == this.walEditsStopOffset) {
275         return false;
276       }
277       WALKey.Builder builder = WALKey.newBuilder();
278       long size = 0;
279       try {
280         long available = -1;
281         try {
282           int firstByte = this.inputStream.read();
283           if (firstByte == -1) {
284             throw new EOFException("First byte is negative");
285           }
286           size = CodedInputStream.readRawVarint32(firstByte, this.inputStream);
287           // available may be < 0 on local fs for instance.  If so, can't depend on it.
288           available = this.inputStream.available();
289           if (available > 0 && available < size) {
290             throw new EOFException("Available stream not enough for edit, " +
291                 "inputStream.available()= " + this.inputStream.available() + ", " +
292                 "entry size= " + size);
293           }
294           ProtobufUtil.mergeFrom(builder, new LimitInputStream(this.inputStream, size),
295             (int)size);
296         } catch (IOException ipbe) {
297           throw (EOFException) new EOFException("Invalid PB, EOF? Ignoring; originalPosition=" +
298             originalPosition + ", currentPosition=" + this.inputStream.getPos() +
299             ", messageSize=" + size + ", currentAvailable=" + available).initCause(ipbe);
300         }
301         if (!builder.isInitialized()) {
302           // TODO: not clear if we should try to recover from corrupt PB that looks semi-legit.
303           //       If we can get the KV count, we could, theoretically, try to get next record.
304           throw new EOFException("Partial PB while reading WAL, " +
305               "probably an unexpected EOF, ignoring");
306         }
307         WALKey walKey = builder.build();
308         entry.getKey().readFieldsFromPb(walKey, this.byteStringUncompressor);
309         if (!walKey.hasFollowingKvCount() || 0 == walKey.getFollowingKvCount()) {
310           LOG.trace("WALKey has no KVs that follow it; trying the next one");
311           continue;
312         }
313         int expectedCells = walKey.getFollowingKvCount();
314         long posBefore = this.inputStream.getPos();
315         try {
316           int actualCells = entry.getEdit().readFromCells(cellDecoder, expectedCells);
317           if (expectedCells != actualCells) {
318             throw new EOFException("Only read " + actualCells); // other info added in catch
319           }
320         } catch (Exception ex) {
321           String posAfterStr = "<unknown>";
322           try {
323             posAfterStr = this.inputStream.getPos() + "";
324           } catch (Throwable t) {
325             LOG.trace("Error getting pos for error message - ignoring", t);
326           }
327           String message = " while reading " + expectedCells + " WAL KVs; started reading at "
328               + posBefore + " and read up to " + posAfterStr;
329           IOException realEofEx = extractHiddenEof(ex);
330           throw (EOFException) new EOFException("EOF " + message).
331               initCause(realEofEx != null ? realEofEx : ex);
332         }
333         if (trailerPresent && this.inputStream.getPos() > this.walEditsStopOffset) {
334           LOG.error("Read WALTrailer while reading WALEdits. hlog: " + this.path
335               + ", inputStream.getPos(): " + this.inputStream.getPos() + ", walEditsStopOffset: "
336               + this.walEditsStopOffset);
337           throw new EOFException("Read WALTrailer while reading WALEdits");
338         }
339       } catch (EOFException eof) {
340         LOG.trace("Encountered a malformed edit, seeking back to last good position in file", eof);
341         // If originalPosition is < 0, it is rubbish and we cannot use it (probably local fs)
342         if (originalPosition < 0) throw eof;
343         // Else restore our position to original location in hope that next time through we will
344         // read successfully.
345         seekOnFs(originalPosition);
346         return false;
347       }
348       return true;
349     }
350   }
351 
352   private IOException extractHiddenEof(Exception ex) {
353     // There are two problems we are dealing with here. Hadoop stream throws generic exception
354     // for EOF, not EOFException; and scanner further hides it inside RuntimeException.
355     IOException ioEx = null;
356     if (ex instanceof EOFException) {
357       return (EOFException)ex;
358     } else if (ex instanceof IOException) {
359       ioEx = (IOException)ex;
360     } else if (ex instanceof RuntimeException
361         && ex.getCause() != null && ex.getCause() instanceof IOException) {
362       ioEx = (IOException)ex.getCause();
363     }
364     if (ioEx != null) {
365       if (ioEx.getMessage().contains("EOF")) return ioEx;
366       return null;
367     }
368     return null;
369   }
370 
371   @Override
372   public WALTrailer getWALTrailer() {
373     return trailer;
374   }
375 
376   @Override
377   protected void seekOnFs(long pos) throws IOException {
378     this.inputStream.seek(pos);
379   }
380 }