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.logging.Log;
26  import org.apache.commons.logging.LogFactory;
27  import org.apache.hadoop.fs.FSDataInputStream;
28  import org.apache.hadoop.fs.FileSystem;
29  import org.apache.hadoop.fs.Path;
30  import org.apache.hadoop.hbase.fs.HFileSystem;
31  import org.apache.hadoop.hbase.io.FileLink;
32  
33  import com.google.common.annotations.VisibleForTesting;
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  public class FSDataInputStreamWrapper {
41    private static final Log LOG = LogFactory.getLog(FSDataInputStreamWrapper.class);
42    private static final boolean isLogTraceEnabled = LOG.isTraceEnabled();
43  
44    private final HFileSystem hfs;
45    private final Path path;
46    private final FileLink link;
47    private final boolean doCloseStreams;
48  
49    /** Two stream handles, one with and one without FS-level checksum.
50     * HDFS checksum setting is on FS level, not single read level, so you have to keep two
51     * FS objects and two handles open to interleave different reads freely, which is very sad.
52     * This is what we do:
53     * 1) First, we need to read the trailer of HFile to determine checksum parameters.
54     *  We always use FS checksum to do that, so ctor opens {@link #stream}.
55     * 2.1) After that, if HBase checksum is not used, we'd just always use {@link #stream};
56     * 2.2) If HBase checksum can be used, we'll open {@link #streamNoFsChecksum},
57     *  and close {@link #stream}. User MUST call prepareForBlockReader for that to happen;
58     *  if they don't, (2.1) will be the default.
59     * 3) The users can call {@link #shouldUseHBaseChecksum()}, and pass its result to
60     *  {@link #getStream(boolean)} to get stream (if Java had out/pointer params we could
61     *  return both in one call). This stream is guaranteed to be set.
62     * 4) The first time HBase checksum fails, one would call {@link #fallbackToFsChecksum(int)}.
63     * That will take lock, and open {@link #stream}. While this is going on, others will
64     * continue to use the old stream; if they also want to fall back, they'll also call
65     * {@link #fallbackToFsChecksum(int)}, and block until {@link #stream} is set.
66     * 5) After some number of checksumOk() calls, we will go back to using HBase checksum.
67     * We will have 2 handles; however we presume checksums fail so rarely that we don't care.
68     */
69    private volatile FSDataInputStream stream = null;
70    private volatile FSDataInputStream streamNoFsChecksum = null;
71    private Object streamNoFsChecksumFirstCreateLock = new Object();
72  
73    // The configuration states that we should validate hbase checksums
74    private boolean useHBaseChecksumConfigured;
75  
76    // Record the current state of this reader with respect to
77    // validating checkums in HBase. This is originally set the same
78    // value as useHBaseChecksumConfigured, but can change state as and when
79    // we encounter checksum verification failures.
80    private volatile boolean useHBaseChecksum;
81  
82    // In the case of a checksum failure, do these many succeeding
83    // reads without hbase checksum verification.
84    private volatile int hbaseChecksumOffCount = -1;
85  
86    private Boolean instanceOfCanUnbuffer = null;
87    // Using reflection to get org.apache.hadoop.fs.CanUnbuffer#unbuffer method to avoid compilation
88    // errors against Hadoop pre 2.6.4 and 2.7.1 versions.
89    private Method unbuffer = null;
90  
91    public FSDataInputStreamWrapper(FileSystem fs, Path path) throws IOException {
92      this(fs, null, path);
93    }
94  
95    public FSDataInputStreamWrapper(FileSystem fs, FileLink link) throws IOException {
96      this(fs, link, null);
97    }
98  
99    private FSDataInputStreamWrapper(FileSystem fs, FileLink link, Path path) throws IOException {
100     assert (path == null) != (link == null);
101     this.path = path;
102     this.link = link;
103     this.doCloseStreams = true;
104     // If the fs is not an instance of HFileSystem, then create an instance of HFileSystem
105     // that wraps over the specified fs. In this case, we will not be able to avoid
106     // checksumming inside the filesystem.
107     this.hfs = (fs instanceof HFileSystem) ? (HFileSystem)fs : new HFileSystem(fs);
108 
109     // Initially we are going to read the tail block. Open the reader w/FS checksum.
110     this.useHBaseChecksumConfigured = this.useHBaseChecksum = false;
111     this.stream = (link != null) ? link.open(hfs) : hfs.open(path);
112   }
113 
114   /**
115    * Prepares the streams for block reader. NOT THREAD SAFE. Must be called once, after any
116    * reads finish and before any other reads start (what happens in reality is we read the
117    * tail, then call this based on what's in the tail, then read blocks).
118    * @param forceNoHBaseChecksum Force not using HBase checksum.
119    */
120   public void prepareForBlockReader(boolean forceNoHBaseChecksum) throws IOException {
121     if (hfs == null) return;
122     assert this.stream != null && !this.useHBaseChecksumConfigured;
123     boolean useHBaseChecksum =
124         !forceNoHBaseChecksum && hfs.useHBaseChecksum() && (hfs.getNoChecksumFs() != hfs);
125 
126     if (useHBaseChecksum) {
127       FileSystem fsNc = hfs.getNoChecksumFs();
128       this.streamNoFsChecksum = (link != null) ? link.open(fsNc) : fsNc.open(path);
129       this.useHBaseChecksumConfigured = this.useHBaseChecksum = useHBaseChecksum;
130       // Close the checksum stream; we will reopen it if we get an HBase checksum failure.
131       this.stream.close();
132       this.stream = null;
133     }
134   }
135 
136   /** For use in tests. */
137   @VisibleForTesting
138   public FSDataInputStreamWrapper(FSDataInputStream fsdis) {
139     this(fsdis, fsdis);
140   }
141 
142   /** For use in tests. */
143   @VisibleForTesting
144   public FSDataInputStreamWrapper(FSDataInputStream fsdis, FSDataInputStream noChecksum) {
145     doCloseStreams = false;
146     stream = fsdis;
147     streamNoFsChecksum = noChecksum;
148     path = null;
149     link = null;
150     hfs = null;
151     useHBaseChecksumConfigured = useHBaseChecksum = false;
152   }
153 
154   /**
155    * @return Whether we are presently using HBase checksum.
156    */
157   public boolean shouldUseHBaseChecksum() {
158     return this.useHBaseChecksum;
159   }
160 
161   /**
162    * Get the stream to use. Thread-safe.
163    * @param useHBaseChecksum must be the value that shouldUseHBaseChecksum has returned
164    *  at some point in the past, otherwise the result is undefined.
165    */
166   public FSDataInputStream getStream(boolean useHBaseChecksum) {
167     return useHBaseChecksum ? this.streamNoFsChecksum : this.stream;
168   }
169 
170   /**
171    * Read from non-checksum stream failed, fall back to FS checksum. Thread-safe.
172    * @param offCount For how many checksumOk calls to turn off the HBase checksum.
173    */
174   public FSDataInputStream fallbackToFsChecksum(int offCount) throws IOException {
175     // checksumOffCount is speculative, but let's try to reset it less.
176     boolean partOfConvoy = false;
177     if (this.stream == null) {
178       synchronized (streamNoFsChecksumFirstCreateLock) {
179         partOfConvoy = (this.stream != null);
180         if (!partOfConvoy) {
181           this.stream = (link != null) ? link.open(hfs) : hfs.open(path);
182         }
183       }
184     }
185     if (!partOfConvoy) {
186       this.useHBaseChecksum = false;
187       this.hbaseChecksumOffCount = offCount;
188     }
189     return this.stream;
190   }
191 
192   /** Report that checksum was ok, so we may ponder going back to HBase checksum. */
193   public void checksumOk() {
194     if (this.useHBaseChecksumConfigured && !this.useHBaseChecksum
195         && (this.hbaseChecksumOffCount-- < 0)) {
196       // The stream we need is already open (because we were using HBase checksum in the past).
197       assert this.streamNoFsChecksum != null;
198       this.useHBaseChecksum = true;
199     }
200   }
201 
202   /** Close stream(s) if necessary. */
203   public void close() throws IOException {
204     if (!doCloseStreams) return;
205     try {
206       if (stream != streamNoFsChecksum && streamNoFsChecksum != null) {
207         streamNoFsChecksum.close();
208         streamNoFsChecksum = null;
209       }
210     } finally {
211       if (stream != null) {
212         stream.close();
213         stream = null;
214       }
215     }
216   }
217 
218   public HFileSystem getHfs() {
219     return this.hfs;
220   }
221 
222   /**
223    * This will free sockets and file descriptors held by the stream only when the stream implements
224    * org.apache.hadoop.fs.CanUnbuffer. NOT THREAD SAFE. Must be called only when all the clients
225    * using this stream to read the blocks have finished reading. If by chance the stream is
226    * unbuffered and there are clients still holding this stream for read then on next client read
227    * request a new socket will be opened by Datanode without client knowing about it and will serve
228    * its read request. Note: If this socket is idle for some time then the DataNode will close the
229    * socket and the socket will move into CLOSE_WAIT state and on the next client request on this
230    * stream, the current socket will be closed and a new socket will be opened to serve the
231    * requests.
232    */
233   @SuppressWarnings({ "rawtypes" })
234   public void unbuffer() {
235     FSDataInputStream stream = this.getStream(this.shouldUseHBaseChecksum());
236     if (stream != null) {
237       InputStream wrappedStream = stream.getWrappedStream();
238       // CanUnbuffer interface was added as part of HDFS-7694 and the fix is available in Hadoop
239       // 2.6.4+ and 2.7.1+ versions only so check whether the stream object implements the
240       // CanUnbuffer interface or not and based on that call the unbuffer api.
241       final Class<? extends InputStream> streamClass = wrappedStream.getClass();
242       if (this.instanceOfCanUnbuffer == null) {
243         // To ensure we compute whether the stream is instance of CanUnbuffer only once.
244         this.instanceOfCanUnbuffer = false;
245         Class<?>[] streamInterfaces = streamClass.getInterfaces();
246         for (Class c : streamInterfaces) {
247           if (c.getCanonicalName().toString().equals("org.apache.hadoop.fs.CanUnbuffer")) {
248             try {
249               this.unbuffer = streamClass.getDeclaredMethod("unbuffer");
250             } catch (NoSuchMethodException | SecurityException e) {
251               if (isLogTraceEnabled) {
252                 LOG.trace("Failed to find 'unbuffer' method in class " + streamClass
253                     + " . So there may be a TCP socket connection "
254                     + "left open in CLOSE_WAIT state.", e);
255               }
256               return;
257             }
258             this.instanceOfCanUnbuffer = true;
259             break;
260           }
261         }
262       }
263       if (this.instanceOfCanUnbuffer) {
264         try {
265           this.unbuffer.invoke(wrappedStream);
266         } catch (IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
267           if (isLogTraceEnabled) {
268             LOG.trace("Failed to invoke 'unbuffer' method in class " + streamClass
269                 + " . So there may be a TCP socket connection left open in CLOSE_WAIT state.", e);
270           }
271         }
272       } else {
273         if (isLogTraceEnabled) {
274           LOG.trace("Failed to find 'unbuffer' method in class " + streamClass
275               + " . So there may be a TCP socket connection "
276               + "left open in CLOSE_WAIT state. For more details check "
277               + "https://issues.apache.org/jira/browse/HBASE-9393");
278         }
279       }
280     }
281   }
282 }