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.IOException;
23  import java.lang.reflect.Field;
24  import java.util.NavigableMap;
25  
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.conf.Configuration;
29  import org.apache.hadoop.fs.FSDataInputStream;
30  import org.apache.hadoop.fs.FileSystem;
31  import org.apache.hadoop.fs.Path;
32  import org.apache.hadoop.hbase.HBaseInterfaceAudience;
33  import org.apache.hadoop.hbase.classification.InterfaceAudience;
34  import org.apache.hadoop.hbase.wal.WAL.Entry;
35  import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
36  import org.apache.hadoop.io.SequenceFile;
37  import org.apache.hadoop.io.SequenceFile.Metadata;
38  import org.apache.hadoop.io.Text;
39  
40  @InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX,
41    HBaseInterfaceAudience.CONFIG})
42  public class SequenceFileLogReader extends ReaderBase {
43    private static final Log LOG = LogFactory.getLog(SequenceFileLogReader.class);
44  
45    // Legacy stuff from pre-PB WAL metadata.
46    private static final Text WAL_VERSION_KEY = new Text("version");
47    // Let the version be 1.  Let absence of a version meta tag be old, version 0.
48    // Set this version '1' to be the version that introduces compression,
49    // the COMPRESSION_VERSION.
50    private static final int COMPRESSION_VERSION = 1;
51    private static final Text WAL_COMPRESSION_TYPE_KEY = new Text("compression.type");
52    private static final Text DICTIONARY_COMPRESSION_TYPE = new Text("dictionary");
53  
54    /**
55     * Hack just to set the correct file length up in SequenceFile.Reader.
56     * See HADOOP-6307.  The below is all about setting the right length on the
57     * file we are reading.  fs.getFileStatus(file).getLen() is passed down to
58     * a private SequenceFile.Reader constructor.  This won't work.  Need to do
59     * the available on the stream.  The below is ugly.  It makes getPos, the
60     * first time its called, return length of the file -- i.e. tell a lie -- just
61     * so this line up in SF.Reader's constructor ends up with right answer:
62     *
63     *         this.end = in.getPos() + length;
64     *
65     */
66    private static class WALReader extends SequenceFile.Reader {
67  
68      WALReader(final FileSystem fs, final Path p, final Configuration c)
69      throws IOException {
70        super(fs, p, c);
71      }
72  
73      @Override
74      protected FSDataInputStream openFile(FileSystem fs, Path file,
75        int bufferSize, long length)
76      throws IOException {
77        return new WALReaderFSDataInputStream(super.openFile(fs, file,
78          bufferSize, length), length);
79      }
80  
81      /**
82       * Override just so can intercept first call to getPos.
83       */
84      static class WALReaderFSDataInputStream extends FSDataInputStream {
85        private boolean firstGetPosInvocation = true;
86        private long length;
87  
88        WALReaderFSDataInputStream(final FSDataInputStream is, final long l)
89        throws IOException {
90          super(is);
91          this.length = l;
92        }
93  
94        // This section can be confusing.  It is specific to how HDFS works.
95        // Let me try to break it down.  This is the problem:
96        //
97        //  1. HDFS DataNodes update the NameNode about a filename's length
98        //     on block boundaries or when a file is closed. Therefore,
99        //     if an RS dies, then the NN's fs.getLength() can be out of date
100       //  2. this.in.available() would work, but it returns int &
101       //     therefore breaks for files > 2GB (happens on big clusters)
102       //  3. DFSInputStream.getFileLength() gets the actual length from the DNs
103       //  4. DFSInputStream is wrapped 2 levels deep : this.in.in
104       //
105       // So, here we adjust getPos() using getFileLength() so the
106       // SequenceFile.Reader constructor (aka: first invocation) comes out
107       // with the correct end of the file:
108       //         this.end = in.getPos() + length;
109       @Override
110       public long getPos() throws IOException {
111         if (this.firstGetPosInvocation) {
112           this.firstGetPosInvocation = false;
113           long adjust = 0;
114           HdfsDataInputStream hdfsDataInputStream = null;
115           try {
116             if (this.in.getClass().getName().endsWith("HdfsDataInputStream")
117                 || this.in.getClass().getName().endsWith("DFSInputStream")) {
118               hdfsDataInputStream = (HdfsDataInputStream) this.getWrappedStream();
119               long realLength = hdfsDataInputStream.getVisibleLength();
120               assert(realLength >= this.length);
121               adjust = realLength - this.length;
122             } else {
123               LOG.info(
124                 "Input stream class: " + this.in.getClass().getName() + ", not adjusting length");
125             }
126           } catch (Exception e) {
127             LOG.warn("Error while trying to get accurate file length.  "
128                 + "Truncation / data loss may occur if RegionServers die.",
129               e);
130             throw new IOException(e);
131           }
132           return adjust + super.getPos();
133         }
134         return super.getPos();
135       }
136     }
137   }
138 
139   // Protected for tests.
140   protected SequenceFile.Reader reader;
141   long entryStart = 0; // needed for logging exceptions
142 
143   public SequenceFileLogReader() {
144     super();
145   }
146 
147   @Override
148   public void close() throws IOException {
149     try {
150       if (reader != null) {
151         this.reader.close();
152         this.reader = null;
153       }
154     } catch (IOException ioe) {
155       throw addFileInfoToException(ioe);
156     }
157   }
158 
159   @Override
160   public long getPosition() throws IOException {
161     return reader != null ? reader.getPosition() : 0;
162   }
163 
164   @Override
165   public void reset() throws IOException {
166     // Resetting the reader lets us see newly added data if the file is being written to
167     // We also keep the same compressionContext which was previously populated for this file
168     reader = new WALReader(fs, path, conf);
169   }
170 
171   @Override
172   protected String initReader(FSDataInputStream stream) throws IOException {
173     // We don't use the stream because we have to have the magic stream above.
174     if (stream != null) {
175       stream.close();
176     }
177     reset();
178     return null;
179   }
180   
181   @Override
182   protected void initAfterCompression(String cellCodecClsName) throws IOException {
183     // Nothing to do here
184   }
185 
186   @Override
187   protected void initAfterCompression() throws IOException {
188     // Nothing to do here
189   }
190 
191   @Override
192   protected boolean hasCompression() {
193     return isWALCompressionEnabled(reader.getMetadata());
194   }
195 
196   @Override
197   protected boolean hasTagCompression() {
198     // Tag compression not supported with old SequenceFileLog Reader/Writer
199     return false;
200   }
201 
202   /**
203    * Call this method after init() has been executed
204    * @return whether WAL compression is enabled
205    */
206   static boolean isWALCompressionEnabled(final Metadata metadata) {
207     // Check version is >= VERSION?
208     Text txt = metadata.get(WAL_VERSION_KEY);
209     if (txt == null || Integer.parseInt(txt.toString()) < COMPRESSION_VERSION) {
210       return false;
211     }
212     // Now check that compression type is present.  Currently only one value.
213     txt = metadata.get(WAL_COMPRESSION_TYPE_KEY);
214     return txt != null && txt.equals(DICTIONARY_COMPRESSION_TYPE);
215   }
216 
217 
218   /**
219    * fill in the passed entry with teh next key/value.
220    * Note that because this format deals with our legacy storage, the provided
221    * Entery MUST use an {@link HLogKey} for the key.
222    * @return boolean indicating if the contents of Entry have been filled in.
223    */
224   @Override
225   protected boolean readNext(Entry e) throws IOException {
226     try {
227       if (!(e.getKey() instanceof HLogKey)) {
228         final IllegalArgumentException exception = new IllegalArgumentException(
229             "SequenceFileLogReader only works when given entries that have HLogKey for keys. This" +
230             " one had '" + e.getKey().getClass() + "'");
231         LOG.error("We need to use the legacy SequenceFileLogReader to handle a " +
232             " pre-0.96 style WAL, but HBase internals failed to use the deprecated HLogKey class." +
233             " This is a bug; please file an issue or email the developer mailing list. You will " +
234             "need the following exception details when seeking help from the HBase community.",
235             exception);
236         throw exception;
237       }
238       boolean hasNext = this.reader.next((HLogKey)e.getKey(), e.getEdit());
239       if (!hasNext) return false;
240       // Scopes are probably in WAL edit, move to key
241       NavigableMap<byte[], Integer> scopes = e.getEdit().getAndRemoveScopes();
242       if (scopes != null) {
243         e.getKey().readOlderScopes(scopes);
244       }
245       return true;
246     } catch (IOException ioe) {
247       throw addFileInfoToException(ioe);
248     }
249   }
250 
251   @Override
252   protected void seekOnFs(long pos) throws IOException {
253     try {
254       reader.seek(pos);
255     } catch (IOException ioe) {
256       throw addFileInfoToException(ioe);
257     }
258   }
259 
260   protected IOException addFileInfoToException(final IOException ioe)
261   throws IOException {
262     long pos = -1;
263     try {
264       pos = getPosition();
265     } catch (IOException e) {
266       LOG.warn("Failed getting position to add to throw", e);
267     }
268 
269     // See what SequenceFile.Reader thinks is the end of the file
270     long end = Long.MAX_VALUE;
271     try {
272       Field fEnd = SequenceFile.Reader.class.getDeclaredField("end");
273       fEnd.setAccessible(true);
274       end = fEnd.getLong(this.reader);
275     } catch(NoSuchFieldException nfe) {
276        /* reflection failure, keep going */
277       if (LOG.isTraceEnabled()) LOG.trace(nfe);
278     } catch(IllegalAccessException iae) {
279        /* reflection failure, keep going */
280       if (LOG.isTraceEnabled()) LOG.trace(iae);
281     } catch(Exception e) {
282        /* All other cases. Should we handle it more aggressively? */
283        LOG.warn("Unexpected exception when accessing the end field", e);
284     }
285  
286     String msg = (this.path == null? "": this.path.toString()) +
287       ", entryStart=" + entryStart + ", pos=" + pos +
288       ((end == Long.MAX_VALUE) ? "" : ", end=" + end) +
289       ", edit=" + this.edit;
290 
291     // Enhance via reflection so we don't change the original class type
292     try {
293       return (IOException) ioe.getClass()
294         .getConstructor(String.class)
295         .newInstance(msg)
296         .initCause(ioe);
297     } catch(NoSuchMethodException nfe) {
298        /* reflection failure, keep going */
299       if (LOG.isTraceEnabled()) LOG.trace(nfe);
300     } catch(IllegalAccessException iae) {
301        /* reflection failure, keep going */
302       if (LOG.isTraceEnabled()) LOG.trace(iae);
303     } catch(Exception e) {
304        /* All other cases. Should we handle it more aggressively? */
305        LOG.warn("Unexpected exception when accessing the end field", e);
306     }
307     return ioe;
308   }
309 }