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  import java.util.NavigableMap;
27  
28  
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.classification.InterfaceAudience;
32  import org.apache.hadoop.conf.Configuration;
33  import org.apache.hadoop.fs.FSDataInputStream;
34  import org.apache.hadoop.fs.FileSystem;
35  import org.apache.hadoop.fs.Path;
36  import org.apache.hadoop.hbase.regionserver.wal.HLog.Entry;
37  import org.apache.hadoop.io.SequenceFile;
38  import org.apache.hadoop.io.Text;
39  import org.apache.hadoop.io.SequenceFile.Metadata;
40  
41  @InterfaceAudience.Private
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 
115           try {
116             Field fIn = FilterInputStream.class.getDeclaredField("in");
117             fIn.setAccessible(true);
118             Object realIn = fIn.get(this.in);
119             // In hadoop 0.22, DFSInputStream is a standalone class.  Before this,
120             // it was an inner class of DFSClient.
121             if (realIn.getClass().getName().endsWith("DFSInputStream")) {
122               Method getFileLength = realIn.getClass().
123                 getDeclaredMethod("getFileLength", new Class<?> []{});
124               getFileLength.setAccessible(true);
125               long realLength = ((Long)getFileLength.
126                 invoke(realIn, new Object []{})).longValue();
127               assert(realLength >= this.length);
128               adjust = realLength - this.length;
129             } else {
130               LOG.info("Input stream class: " + realIn.getClass().getName() +
131                   ", not adjusting length");
132             }
133           } catch(Exception e) {
134             SequenceFileLogReader.LOG.warn(
135               "Error while trying to get accurate file length.  " +
136               "Truncation / data loss may occur if RegionServers die.", e);
137           }
138 
139           return adjust + super.getPos();
140         }
141         return super.getPos();
142       }
143     }
144   }
145 
146   // Protected for tests.
147   protected SequenceFile.Reader reader;
148   long entryStart = 0; // needed for logging exceptions
149 
150   public SequenceFileLogReader() {
151     super();
152   }
153 
154   @Override
155   public void close() throws IOException {
156     try {
157       if (reader != null) {
158         this.reader.close();
159         this.reader = null;
160       }
161     } catch (IOException ioe) {
162       throw addFileInfoToException(ioe);
163     }
164   }
165 
166   @Override
167   public long getPosition() throws IOException {
168     return reader != null ? reader.getPosition() : 0;
169   }
170 
171   @Override
172   public void reset() throws IOException {
173     // Resetting the reader lets us see newly added data if the file is being written to
174     // We also keep the same compressionContext which was previously populated for this file
175     reader = new WALReader(fs, path, conf);
176   }
177 
178   @Override
179   protected void initReader(FSDataInputStream stream) throws IOException {
180     // We don't use the stream because we have to have the magic stream above.
181     if (stream != null) {
182       stream.close();
183     }
184     reset();
185   }
186   
187   @Override
188   protected void initAfterCompression() throws IOException {
189     // Nothing to do here
190   }
191 
192   @Override
193   protected boolean hasCompression() {
194     return isWALCompressionEnabled(reader.getMetadata());
195   }
196 
197   /**
198    * Call this method after init() has been executed
199    * @return whether WAL compression is enabled
200    */
201   static boolean isWALCompressionEnabled(final Metadata metadata) {
202     // Check version is >= VERSION?
203     Text txt = metadata.get(WAL_VERSION_KEY);
204     if (txt == null || Integer.parseInt(txt.toString()) < COMPRESSION_VERSION) {
205       return false;
206     }
207     // Now check that compression type is present.  Currently only one value.
208     txt = metadata.get(WAL_COMPRESSION_TYPE_KEY);
209     return txt != null && txt.equals(DICTIONARY_COMPRESSION_TYPE);
210   }
211 
212 
213   @Override
214   protected boolean readNext(Entry e) throws IOException {
215     try {
216       boolean hasNext = this.reader.next(e.getKey(), e.getEdit());
217       if (!hasNext) return false;
218       // Scopes are probably in WAL edit, move to key
219       NavigableMap<byte[], Integer> scopes = e.getEdit().getAndRemoveScopes();
220       if (scopes != null) {
221         e.getKey().setScopes(scopes);
222       }
223       return true;
224     } catch (IOException ioe) {
225       throw addFileInfoToException(ioe);
226     }
227   }
228 
229   @Override
230   protected void seekOnFs(long pos) throws IOException {
231     try {
232       reader.seek(pos);
233     } catch (IOException ioe) {
234       throw addFileInfoToException(ioe);
235     }
236   }
237 
238   protected IOException addFileInfoToException(final IOException ioe)
239   throws IOException {
240     long pos = -1;
241     try {
242       pos = getPosition();
243     } catch (IOException e) {
244       LOG.warn("Failed getting position to add to throw", e);
245     }
246 
247     // See what SequenceFile.Reader thinks is the end of the file
248     long end = Long.MAX_VALUE;
249     try {
250       Field fEnd = SequenceFile.Reader.class.getDeclaredField("end");
251       fEnd.setAccessible(true);
252       end = fEnd.getLong(this.reader);
253     } catch(Exception e) { /* reflection fail. keep going */ }
254 
255     String msg = (this.path == null? "": this.path.toString()) +
256       ", entryStart=" + entryStart + ", pos=" + pos +
257       ((end == Long.MAX_VALUE) ? "" : ", end=" + end) +
258       ", edit=" + this.edit;
259 
260     // Enhance via reflection so we don't change the original class type
261     try {
262       return (IOException) ioe.getClass()
263         .getConstructor(String.class)
264         .newInstance(msg)
265         .initCause(ioe);
266     } catch(Exception e) { /* reflection fail. keep going */ }
267 
268     return ioe;
269   }
270 }