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.FilterInputStream;
23  import java.io.IOException;
24  import java.lang.reflect.Field;
25  import java.lang.reflect.Method;
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.conf.Configuration;
31  import org.apache.hadoop.fs.FSDataInputStream;
32  import org.apache.hadoop.fs.FileSystem;
33  import org.apache.hadoop.fs.Path;
34  import org.apache.hadoop.io.SequenceFile;
35  
36  @InterfaceAudience.Private
37  public class SequenceFileLogReader implements HLog.Reader {
38    private static final Log LOG = LogFactory.getLog(SequenceFileLogReader.class);
39  
40    /**
41     * Hack just to set the correct file length up in SequenceFile.Reader.
42     * See HADOOP-6307.  The below is all about setting the right length on the
43     * file we are reading.  fs.getFileStatus(file).getLen() is passed down to
44     * a private SequenceFile.Reader constructor.  This won't work.  Need to do
45     * the available on the stream.  The below is ugly.  It makes getPos, the
46     * first time its called, return length of the file -- i.e. tell a lie -- just
47     * so this line up in SF.Reader's constructor ends up with right answer:
48     *
49     *         this.end = in.getPos() + length;
50     *
51     */
52    static class WALReader extends SequenceFile.Reader {
53  
54      WALReader(final FileSystem fs, final Path p, final Configuration c)
55      throws IOException {
56        super(fs, p, c);
57      }
58  
59      @Override
60      protected FSDataInputStream openFile(FileSystem fs, Path file,
61        int bufferSize, long length)
62      throws IOException {
63        return new WALReaderFSDataInputStream(super.openFile(fs, file,
64          bufferSize, length), length);
65      }
66  
67      /**
68       * Call this method after init() has been executed
69       * 
70       * @return whether WAL compression is enabled
71       */
72      public boolean isWALCompressionEnabled() {
73        return SequenceFileLogWriter.isWALCompressionEnabled(this.getMetadata());
74      }
75  
76      /**
77       * Override just so can intercept first call to getPos.
78       */
79      static class WALReaderFSDataInputStream extends FSDataInputStream {
80        private boolean firstGetPosInvocation = true;
81        private long length;
82  
83        WALReaderFSDataInputStream(final FSDataInputStream is, final long l)
84        throws IOException {
85          super(is);
86          this.length = l;
87        }
88  
89        // This section can be confusing.  It is specific to how HDFS works.
90        // Let me try to break it down.  This is the problem:
91        //
92        //  1. HDFS DataNodes update the NameNode about a filename's length
93        //     on block boundaries or when a file is closed. Therefore,
94        //     if an RS dies, then the NN's fs.getLength() can be out of date
95        //  2. this.in.available() would work, but it returns int &
96        //     therefore breaks for files > 2GB (happens on big clusters)
97        //  3. DFSInputStream.getFileLength() gets the actual length from the DNs
98        //  4. DFSInputStream is wrapped 2 levels deep : this.in.in
99        //
100       // So, here we adjust getPos() using getFileLength() so the
101       // SequenceFile.Reader constructor (aka: first invocation) comes out
102       // with the correct end of the file:
103       //         this.end = in.getPos() + length;
104       @Override
105       public long getPos() throws IOException {
106         if (this.firstGetPosInvocation) {
107           this.firstGetPosInvocation = false;
108           long adjust = 0;
109 
110           try {
111             Field fIn = FilterInputStream.class.getDeclaredField("in");
112             fIn.setAccessible(true);
113             Object realIn = fIn.get(this.in);
114             // In hadoop 0.22, DFSInputStream is a standalone class.  Before this,
115             // it was an inner class of DFSClient.
116             if (realIn.getClass().getName().endsWith("DFSInputStream")) {
117               Method getFileLength = realIn.getClass().
118                 getDeclaredMethod("getFileLength", new Class<?> []{});
119               getFileLength.setAccessible(true);
120               long realLength = ((Long)getFileLength.
121                 invoke(realIn, new Object []{})).longValue();
122               assert(realLength >= this.length);
123               adjust = realLength - this.length;
124             } else {
125               LOG.info("Input stream class: " + realIn.getClass().getName() +
126                   ", not adjusting length");
127             }
128           } catch(Exception e) {
129             SequenceFileLogReader.LOG.warn(
130               "Error while trying to get accurate file length.  " +
131               "Truncation / data loss may occur if RegionServers die.", e);
132           }
133 
134           return adjust + super.getPos();
135         }
136         return super.getPos();
137       }
138     }
139   }
140 
141   Configuration conf;
142   WALReader reader;
143   FileSystem fs;
144 
145   // Needed logging exceptions
146   Path path;
147   int edit = 0;
148   long entryStart = 0;
149   boolean emptyCompressionContext = true;
150   /**
151    * Compression context to use reading.  Can be null if no compression.
152    */
153   protected CompressionContext compressionContext = null;
154 
155   protected Class<? extends HLogKey> keyClass;
156 
157   /**
158    * Default constructor.
159    */
160   public SequenceFileLogReader() {
161   }
162 
163   /**
164    * This constructor allows a specific HLogKey implementation to override that
165    * which would otherwise be chosen via configuration property.
166    *
167    * @param keyClass
168    */
169   public SequenceFileLogReader(Class<? extends HLogKey> keyClass) {
170     this.keyClass = keyClass;
171   }
172 
173   @Override
174   public void init(FileSystem fs, Path path, Configuration conf)
175       throws IOException {
176     this.conf = conf;
177     this.path = path;
178     reader = new WALReader(fs, path, conf);
179     this.fs = fs;
180 
181     // If compression is enabled, new dictionaries are created here.
182     boolean compression = reader.isWALCompressionEnabled();
183     if (compression) {
184       try {
185         if (compressionContext == null) {
186           compressionContext = new CompressionContext(LRUDictionary.class);
187         } else {
188           compressionContext.clear();
189         }
190       } catch (Exception e) {
191         throw new IOException("Failed to initialize CompressionContext", e);
192       }
193     }
194   }
195 
196   @Override
197   public void close() throws IOException {
198     try {
199       if (reader != null) {
200         this.reader.close();
201         this.reader = null;
202       }
203     } catch (IOException ioe) {
204       throw addFileInfoToException(ioe);
205     }
206   }
207 
208   @Override
209   public HLog.Entry next() throws IOException {
210     return next(null);
211   }
212 
213   @Override
214   public HLog.Entry next(HLog.Entry reuse) throws IOException {
215     this.entryStart = this.reader.getPosition();
216     HLog.Entry e = reuse;
217     if (e == null) {
218       HLogKey key;
219       if (keyClass == null) {
220         key = HLogUtil.newKey(conf);
221       } else {
222         try {
223           key = keyClass.newInstance();
224         } catch (InstantiationException ie) {
225           throw new IOException(ie);
226         } catch (IllegalAccessException iae) {
227           throw new IOException(iae);
228         }
229       }
230 
231       WALEdit val = new WALEdit();
232       e = new HLog.Entry(key, val);
233     }
234     boolean b = false;
235     try {
236       if (compressionContext != null) {
237         e.setCompressionContext(compressionContext);
238       }
239       b = this.reader.next(e.getKey(), e.getEdit());
240     } catch (IOException ioe) {
241       throw addFileInfoToException(ioe);
242     }
243     edit++;
244     if (compressionContext != null && emptyCompressionContext) {
245       emptyCompressionContext = false;
246     }
247     return b? e: null;
248   }
249 
250   @Override
251   public void seek(long pos) throws IOException {
252     if (compressionContext != null && emptyCompressionContext) {
253       while (next() != null) {
254         if (getPosition() == pos) {
255           emptyCompressionContext = false;
256           break;
257         }
258       }
259     }
260     try {
261       reader.seek(pos);
262     } catch (IOException ioe) {
263       throw addFileInfoToException(ioe);
264     }
265   }
266 
267   @Override
268   public long getPosition() throws IOException {
269     return reader != null ? reader.getPosition() : 0;
270   }
271 
272   protected IOException addFileInfoToException(final IOException ioe)
273   throws IOException {
274     long pos = -1;
275     try {
276       pos = getPosition();
277     } catch (IOException e) {
278       LOG.warn("Failed getting position to add to throw", e);
279     }
280 
281     // See what SequenceFile.Reader thinks is the end of the file
282     long end = Long.MAX_VALUE;
283     try {
284       Field fEnd = SequenceFile.Reader.class.getDeclaredField("end");
285       fEnd.setAccessible(true);
286       end = fEnd.getLong(this.reader);
287     } catch(Exception e) { /* reflection fail. keep going */ }
288 
289     String msg = (this.path == null? "": this.path.toString()) +
290       ", entryStart=" + entryStart + ", pos=" + pos +
291       ((end == Long.MAX_VALUE) ? "" : ", end=" + end) +
292       ", edit=" + this.edit;
293 
294     // Enhance via reflection so we don't change the original class type
295     try {
296       return (IOException) ioe.getClass()
297         .getConstructor(String.class)
298         .newInstance(msg)
299         .initCause(ioe);
300     } catch(Exception e) { /* reflection fail. keep going */ }
301 
302     return ioe;
303   }
304 
305   @Override
306   public void reset() throws IOException {
307     // Resetting the reader lets us see newly added data if the file is being written to
308     // We also keep the same compressionContext which was previously populated for this file
309     reader = new WALReader(fs, path, conf);
310   }
311 }