View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.io;
19  
20  import java.io.IOException;
21  import java.io.InputStream;
22  import java.lang.reflect.InvocationTargetException;
23  import java.lang.reflect.Method;
24  
25  import org.apache.commons.io.IOUtils;
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.fs.FSDataInputStream;
29  import org.apache.hadoop.fs.FileSystem;
30  import org.apache.hadoop.fs.Path;
31  import org.apache.hadoop.hbase.classification.InterfaceAudience;
32  import org.apache.hadoop.hbase.fs.HFileSystem;
33  import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
34  
35  /**
36   * Wrapper for input stream(s) that takes care of the interaction of FS and HBase checksums,
37   * as well as closing streams. Initialization is not thread-safe, but normal operation is;
38   * see method comments.
39   */
40  @InterfaceAudience.Private
41  public class FSDataInputStreamWrapper {
42    private static final Log LOG = LogFactory.getLog(FSDataInputStreamWrapper.class);
43    private static final boolean isLogTraceEnabled = LOG.isTraceEnabled();
44  
45    private final HFileSystem hfs;
46    private final Path path;
47    private final FileLink link;
48    private final boolean doCloseStreams;
49  
50    /** Two stream handles, one with and one without FS-level checksum.
51     * HDFS checksum setting is on FS level, not single read level, so you have to keep two
52     * FS objects and two handles open to interleave different reads freely, which is very sad.
53     * This is what we do:
54     * 1) First, we need to read the trailer of HFile to determine checksum parameters.
55     *  We always use FS checksum to do that, so ctor opens {@link #stream}.
56     * 2.1) After that, if HBase checksum is not used, we'd just always use {@link #stream};
57     * 2.2) If HBase checksum can be used, we'll open {@link #streamNoFsChecksum},
58     *  and close {@link #stream}. User MUST call prepareForBlockReader for that to happen;
59     *  if they don't, (2.1) will be the default.
60     * 3) The users can call {@link #shouldUseHBaseChecksum()}, and pass its result to
61     *  {@link #getStream(boolean)} to get stream (if Java had out/pointer params we could
62     *  return both in one call). This stream is guaranteed to be set.
63     * 4) The first time HBase checksum fails, one would call {@link #fallbackToFsChecksum(int)}.
64     * That will take lock, and open {@link #stream}. While this is going on, others will
65     * continue to use the old stream; if they also want to fall back, they'll also call
66     * {@link #fallbackToFsChecksum(int)}, and block until {@link #stream} is set.
67     * 5) After some number of checksumOk() calls, we will go back to using HBase checksum.
68     * We will have 2 handles; however we presume checksums fail so rarely that we don't care.
69     */
70    private volatile FSDataInputStream stream = null;
71    private volatile FSDataInputStream streamNoFsChecksum = null;
72    private final Object streamNoFsChecksumFirstCreateLock = new Object();
73  
74    // The configuration states that we should validate hbase checksums
75    private boolean useHBaseChecksumConfigured;
76  
77    // Record the current state of this reader with respect to
78    // validating checkums in HBase. This is originally set the same
79    // value as useHBaseChecksumConfigured, but can change state as and when
80    // we encounter checksum verification failures.
81    private volatile boolean useHBaseChecksum;
82  
83    // In the case of a checksum failure, do these many succeeding
84    // reads without hbase checksum verification.
85    private volatile int hbaseChecksumOffCount = -1;
86  
87    private Boolean instanceOfCanUnbuffer = null;
88    // Using reflection to get org.apache.hadoop.fs.CanUnbuffer#unbuffer method to avoid compilation
89    // errors against Hadoop pre 2.6.4 and 2.7.1 versions.
90    private Method unbuffer = null;
91  
92    private final static ReadStatistics readStatistics = new ReadStatistics();
93  
94    private static class ReadStatistics {
95      long totalBytesRead;
96      long totalLocalBytesRead;
97      long totalShortCircuitBytesRead;
98      long totalZeroCopyBytesRead;
99    }
100 
101   public FSDataInputStreamWrapper(FileSystem fs, Path path) throws IOException {
102     this(fs, null, path, false);
103   }
104 
105   public FSDataInputStreamWrapper(FileSystem fs, Path path, boolean dropBehind) throws IOException {
106     this(fs, null, path, dropBehind);
107   }
108 
109   public FSDataInputStreamWrapper(FileSystem fs, FileLink link) throws IOException {
110     this(fs, link, null, false);
111   }
112   public FSDataInputStreamWrapper(FileSystem fs, FileLink link,
113                                   boolean dropBehind) throws IOException {
114     this(fs, link, null, dropBehind);
115   }
116 
117   private FSDataInputStreamWrapper(FileSystem fs, FileLink link,
118                                    Path path, boolean dropBehind) throws IOException {
119     assert (path == null) != (link == null);
120     this.path = path;
121     this.link = link;
122     this.doCloseStreams = true;
123     // If the fs is not an instance of HFileSystem, then create an instance of HFileSystem
124     // that wraps over the specified fs. In this case, we will not be able to avoid
125     // checksumming inside the filesystem.
126     this.hfs = (fs instanceof HFileSystem) ? (HFileSystem)fs : new HFileSystem(fs);
127 
128     // Initially we are going to read the tail block. Open the reader w/FS checksum.
129     this.useHBaseChecksumConfigured = this.useHBaseChecksum = false;
130     this.stream = (link != null) ? link.open(hfs) : hfs.open(path);
131     try {
132       this.stream.setDropBehind(dropBehind);
133     } catch (Exception e) {
134       // Skipped.
135     }
136   }
137 
138 
139   /**
140    * Prepares the streams for block reader. NOT THREAD SAFE. Must be called once, after any
141    * reads finish and before any other reads start (what happens in reality is we read the
142    * tail, then call this based on what's in the tail, then read blocks).
143    * @param forceNoHBaseChecksum Force not using HBase checksum.
144    */
145   public void prepareForBlockReader(boolean forceNoHBaseChecksum) throws IOException {
146     if (hfs == null) return;
147     assert this.stream != null && !this.useHBaseChecksumConfigured;
148     boolean useHBaseChecksum =
149         !forceNoHBaseChecksum && hfs.useHBaseChecksum() && (hfs.getNoChecksumFs() != hfs);
150 
151     if (useHBaseChecksum) {
152       FileSystem fsNc = hfs.getNoChecksumFs();
153       this.streamNoFsChecksum = (link != null) ? link.open(fsNc) : fsNc.open(path);
154       this.useHBaseChecksumConfigured = this.useHBaseChecksum = useHBaseChecksum;
155       // Close the checksum stream; we will reopen it if we get an HBase checksum failure.
156       this.stream.close();
157       this.stream = null;
158     }
159   }
160 
161   /** For use in tests. */
162   public FSDataInputStreamWrapper(FSDataInputStream fsdis) {
163     this(fsdis, fsdis);
164   }
165 
166   /** For use in tests. */
167   public FSDataInputStreamWrapper(FSDataInputStream fsdis, FSDataInputStream noChecksum) {
168     doCloseStreams = false;
169     stream = fsdis;
170     streamNoFsChecksum = noChecksum;
171     path = null;
172     link = null;
173     hfs = null;
174     useHBaseChecksumConfigured = useHBaseChecksum = false;
175   }
176 
177   /**
178    * @return Whether we are presently using HBase checksum.
179    */
180   public boolean shouldUseHBaseChecksum() {
181     return this.useHBaseChecksum;
182   }
183 
184   /**
185    * Get the stream to use. Thread-safe.
186    * @param useHBaseChecksum must be the value that shouldUseHBaseChecksum has returned
187    *  at some point in the past, otherwise the result is undefined.
188    */
189   public FSDataInputStream getStream(boolean useHBaseChecksum) {
190     return useHBaseChecksum ? this.streamNoFsChecksum : this.stream;
191   }
192 
193   /**
194    * Read from non-checksum stream failed, fall back to FS checksum. Thread-safe.
195    * @param offCount For how many checksumOk calls to turn off the HBase checksum.
196    */
197   public FSDataInputStream fallbackToFsChecksum(int offCount) throws IOException {
198     // checksumOffCount is speculative, but let's try to reset it less.
199     boolean partOfConvoy = false;
200     if (this.stream == null) {
201       synchronized (streamNoFsChecksumFirstCreateLock) {
202         partOfConvoy = (this.stream != null);
203         if (!partOfConvoy) {
204           this.stream = (link != null) ? link.open(hfs) : hfs.open(path);
205         }
206       }
207     }
208     if (!partOfConvoy) {
209       this.useHBaseChecksum = false;
210       this.hbaseChecksumOffCount = offCount;
211     }
212     return this.stream;
213   }
214 
215   /** Report that checksum was ok, so we may ponder going back to HBase checksum. */
216   public void checksumOk() {
217     if (this.useHBaseChecksumConfigured && !this.useHBaseChecksum
218         && (this.hbaseChecksumOffCount-- < 0)) {
219       // The stream we need is already open (because we were using HBase checksum in the past).
220       assert this.streamNoFsChecksum != null;
221       this.useHBaseChecksum = true;
222     }
223   }
224 
225   private void updateInputStreamStatistics(FSDataInputStream stream) {
226     // If the underlying file system is HDFS, update read statistics upon close.
227     if (stream instanceof HdfsDataInputStream) {
228       /**
229        * Because HDFS ReadStatistics is calculated per input stream, it is not
230        * feasible to update the aggregated number in real time. Instead, the
231        * metrics are updated when an input stream is closed.
232        */
233       HdfsDataInputStream hdfsDataInputStream = (HdfsDataInputStream)stream;
234       synchronized (readStatistics) {
235         readStatistics.totalBytesRead += hdfsDataInputStream.getReadStatistics().
236           getTotalBytesRead();
237         readStatistics.totalLocalBytesRead += hdfsDataInputStream.getReadStatistics().
238           getTotalLocalBytesRead();
239         readStatistics.totalShortCircuitBytesRead += hdfsDataInputStream.getReadStatistics().
240           getTotalShortCircuitBytesRead();
241         readStatistics.totalZeroCopyBytesRead += hdfsDataInputStream.getReadStatistics().
242           getTotalZeroCopyBytesRead();
243       }
244     }
245   }
246 
247   public static long getTotalBytesRead() {
248     synchronized (readStatistics) {
249       return readStatistics.totalBytesRead;
250     }
251   }
252 
253   public static long getLocalBytesRead() {
254     synchronized (readStatistics) {
255       return readStatistics.totalLocalBytesRead;
256     }
257   }
258 
259   public static long getShortCircuitBytesRead() {
260     synchronized (readStatistics) {
261       return readStatistics.totalShortCircuitBytesRead;
262     }
263   }
264 
265   public static long getZeroCopyBytesRead() {
266     synchronized (readStatistics) {
267       return readStatistics.totalZeroCopyBytesRead;
268     }
269   }
270 
271   /** Close stream(s) if necessary. */
272   public void close() {
273     if (!doCloseStreams) {
274       return;
275     }
276     updateInputStreamStatistics(this.streamNoFsChecksum);
277     // we do not care about the close exception as it is for reading, no data loss issue.
278     IOUtils.closeQuietly(streamNoFsChecksum);
279     updateInputStreamStatistics(stream);
280     IOUtils.closeQuietly(stream);
281   }
282 
283   public HFileSystem getHfs() {
284     return this.hfs;
285   }
286 
287   /**
288    * This will free sockets and file descriptors held by the stream only when the stream implements
289    * org.apache.hadoop.fs.CanUnbuffer. NOT THREAD SAFE. Must be called only when all the clients
290    * using this stream to read the blocks have finished reading. If by chance the stream is
291    * unbuffered and there are clients still holding this stream for read then on next client read
292    * request a new socket will be opened by Datanode without client knowing about it and will serve
293    * its read request. Note: If this socket is idle for some time then the DataNode will close the
294    * socket and the socket will move into CLOSE_WAIT state and on the next client request on this
295    * stream, the current socket will be closed and a new socket will be opened to serve the
296    * requests.
297    */
298   @SuppressWarnings({ "rawtypes" })
299   public void unbuffer() {
300     FSDataInputStream stream = this.getStream(this.shouldUseHBaseChecksum());
301     if (stream != null) {
302       InputStream wrappedStream = stream.getWrappedStream();
303       // CanUnbuffer interface was added as part of HDFS-7694 and the fix is available in Hadoop
304       // 2.6.4+ and 2.7.1+ versions only so check whether the stream object implements the
305       // CanUnbuffer interface or not and based on that call the unbuffer api.
306       final Class<? extends InputStream> streamClass = wrappedStream.getClass();
307       if (this.instanceOfCanUnbuffer == null) {
308         // To ensure we compute whether the stream is instance of CanUnbuffer only once.
309         this.instanceOfCanUnbuffer = false;
310         Class<?>[] streamInterfaces = streamClass.getInterfaces();
311         for (Class c : streamInterfaces) {
312           if (c.getCanonicalName().toString().equals("org.apache.hadoop.fs.CanUnbuffer")) {
313             try {
314               this.unbuffer = streamClass.getDeclaredMethod("unbuffer");
315             } catch (NoSuchMethodException | SecurityException e) {
316               if (isLogTraceEnabled) {
317                 LOG.trace("Failed to find 'unbuffer' method in class " + streamClass
318                     + " . So there may be a TCP socket connection "
319                     + "left open in CLOSE_WAIT state.", e);
320               }
321               return;
322             }
323             this.instanceOfCanUnbuffer = true;
324             break;
325           }
326         }
327       }
328       if (this.instanceOfCanUnbuffer) {
329         try {
330           this.unbuffer.invoke(wrappedStream);
331         } catch (IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
332           if (isLogTraceEnabled) {
333             LOG.trace("Failed to invoke 'unbuffer' method in class " + streamClass
334                 + " . So there may be a TCP socket connection left open in CLOSE_WAIT state.", e);
335           }
336         }
337       } else {
338         if (isLogTraceEnabled) {
339           LOG.trace("Failed to find 'unbuffer' method in class " + streamClass
340               + " . So there may be a TCP socket connection "
341               + "left open in CLOSE_WAIT state. For more details check "
342               + "https://issues.apache.org/jira/browse/HBASE-9393");
343         }
344       }
345     }
346   }
347 }