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.lang.reflect.Method;
22  
23  import org.apache.commons.logging.Log;
24  import org.apache.commons.logging.LogFactory;
25  import org.apache.hadoop.fs.FSDataInputStream;
26  import org.apache.hadoop.fs.FileSystem;
27  import org.apache.hadoop.fs.Path;
28  import org.apache.hadoop.hbase.fs.HFileSystem;
29  
30  import com.google.common.annotations.VisibleForTesting;
31  
32  /**
33   * Wrapper for input stream(s) that takes care of the interaction of FS and HBase checksums,
34   * as well as closing streams. Initialization is not thread-safe, but normal operation is;
35   * see method comments.
36   */
37  public class FSDataInputStreamWrapper {
38    static final Log LOG = LogFactory.getLog(FSDataInputStreamWrapper.class);
39  
40    private final HFileSystem hfs;
41    private final Path path;
42    private final FileLink link;
43    private final boolean doCloseStreams;
44  
45    /** Two stream handles, one with and one without FS-level checksum.
46     * HDFS checksum setting is on FS level, not single read level, so you have to keep two
47     * FS objects and two handles open to interleave different reads freely, which is very sad.
48     * This is what we do:
49     * 1) First, we need to read the trailer of HFile to determine checksum parameters.
50     *  We always use FS checksum to do that, so ctor opens {@link #stream}.
51     * 2.1) After that, if HBase checksum is not used, we'd just always use {@link #stream};
52     * 2.2) If HBase checksum can be used, we'll open {@link #streamNoFsChecksum},
53     *  and close {@link #stream}. User MUST call prepareForBlockReader for that to happen;
54     *  if they don't, (2.1) will be the default.
55     * 3) The users can call {@link #shouldUseHBaseChecksum()}, and pass its result to
56     *  {@link #getStream(boolean)} to get stream (if Java had out/pointer params we could
57     *  return both in one call). This stream is guaranteed to be set.
58     * 4) The first time HBase checksum fails, one would call {@link #fallbackToFsChecksum(int)}.
59     * That will take lock, and open {@link #stream}. While this is going on, others will
60     * continue to use the old stream; if they also want to fall back, they'll also call
61     * {@link #fallbackToFsChecksum(int)}, and block until {@link #stream} is set.
62     * 5) After some number of checksumOk() calls, we will go back to using HBase checksum.
63     * We will have 2 handles; however we presume checksums fail so rarely that we don't care.
64     */
65    private volatile FSDataInputStream stream = null;
66    private volatile FSDataInputStream streamNoFsChecksum = null;
67    private Object streamNoFsChecksumFirstCreateLock = new Object();
68  
69    // The configuration states that we should validate hbase checksums
70    private boolean useHBaseChecksumConfigured;
71  
72    // Record the current state of this reader with respect to
73    // validating checkums in HBase. This is originally set the same
74    // value as useHBaseChecksumConfigured, but can change state as and when
75    // we encounter checksum verification failures.
76    private volatile boolean useHBaseChecksum;
77  
78    // In the case of a checksum failure, do these many succeeding
79    // reads without hbase checksum verification.
80    private volatile int hbaseChecksumOffCount = -1;
81  
82    public FSDataInputStreamWrapper(FileSystem fs, Path path) throws IOException {
83      this(fs, null, path, false);
84    }
85  
86    public FSDataInputStreamWrapper(FileSystem fs, Path path, boolean dropBehind) throws IOException {
87      this(fs, null, path, dropBehind);
88    }
89  
90    public FSDataInputStreamWrapper(FileSystem fs, FileLink link) throws IOException {
91      this(fs, link, null, false);
92    }
93    public FSDataInputStreamWrapper(FileSystem fs, FileLink link,
94                                    boolean dropBehind) throws IOException {
95      this(fs, link, null, dropBehind);
96    }
97  
98    private FSDataInputStreamWrapper(FileSystem fs, FileLink link,
99                                     Path path, boolean dropBehind) 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     try {
113       Class<? extends FSDataInputStream> inputStreamClass = this.stream.getClass();
114       try {
115         Method m = inputStreamClass.getDeclaredMethod("setDropBehind",
116           new Class[] { boolean.class });
117         m.invoke(stream, new Object[] { dropBehind });
118       } catch (NoSuchMethodException e) {
119         // Not supported, we can just ignore it
120       } catch (Exception e) {
121         if (LOG.isDebugEnabled()) {
122           LOG.debug("Failed to invoke input stream's setDropBehind method, continuing");
123         }
124       }
125     } catch (Exception e) {
126       // Skipped.
127     }
128   }
129 
130 
131   /**
132    * Prepares the streams for block reader. NOT THREAD SAFE. Must be called once, after any
133    * reads finish and before any other reads start (what happens in reality is we read the
134    * tail, then call this based on what's in the tail, then read blocks).
135    * @param forceNoHBaseChecksum Force not using HBase checksum.
136    */
137   public void prepareForBlockReader(boolean forceNoHBaseChecksum) throws IOException {
138     if (hfs == null) return;
139     assert this.stream != null && !this.useHBaseChecksumConfigured;
140     boolean useHBaseChecksum =
141         !forceNoHBaseChecksum && hfs.useHBaseChecksum() && (hfs.getNoChecksumFs() != hfs);
142 
143     if (useHBaseChecksum) {
144       FileSystem fsNc = hfs.getNoChecksumFs();
145       this.streamNoFsChecksum = (link != null) ? link.open(fsNc) : fsNc.open(path);
146       this.useHBaseChecksumConfigured = this.useHBaseChecksum = useHBaseChecksum;
147       // Close the checksum stream; we will reopen it if we get an HBase checksum failure.
148       this.stream.close();
149       this.stream = null;
150     }
151   }
152 
153   /** For use in tests. */
154   @VisibleForTesting
155   public FSDataInputStreamWrapper(FSDataInputStream fsdis) {
156     this(fsdis, fsdis);
157   }
158 
159   /** For use in tests. */
160   @VisibleForTesting
161   public FSDataInputStreamWrapper(FSDataInputStream fsdis, FSDataInputStream noChecksum) {
162     doCloseStreams = false;
163     stream = fsdis;
164     streamNoFsChecksum = noChecksum;
165     path = null;
166     link = null;
167     hfs = null;
168     useHBaseChecksumConfigured = useHBaseChecksum = false;
169   }
170 
171   /**
172    * @return Whether we are presently using HBase checksum.
173    */
174   public boolean shouldUseHBaseChecksum() {
175     return this.useHBaseChecksum;
176   }
177 
178   /**
179    * Get the stream to use. Thread-safe.
180    * @param useHBaseChecksum must be the value that shouldUseHBaseChecksum has returned
181    *  at some point in the past, otherwise the result is undefined.
182    */
183   public FSDataInputStream getStream(boolean useHBaseChecksum) {
184     return useHBaseChecksum ? this.streamNoFsChecksum : this.stream;
185   }
186 
187   /**
188    * Read from non-checksum stream failed, fall back to FS checksum. Thread-safe.
189    * @param offCount For how many checksumOk calls to turn off the HBase checksum.
190    */
191   public FSDataInputStream fallbackToFsChecksum(int offCount) throws IOException {
192     // checksumOffCount is speculative, but let's try to reset it less.
193     boolean partOfConvoy = false;
194     if (this.stream == null) {
195       synchronized (streamNoFsChecksumFirstCreateLock) {
196         partOfConvoy = (this.stream != null);
197         if (!partOfConvoy) {
198           this.stream = (link != null) ? link.open(hfs) : hfs.open(path);
199         }
200       }
201     }
202     if (!partOfConvoy) {
203       this.useHBaseChecksum = false;
204       this.hbaseChecksumOffCount = offCount;
205     }
206     return this.stream;
207   }
208 
209   /** Report that checksum was ok, so we may ponder going back to HBase checksum. */
210   public void checksumOk() {
211     if (this.useHBaseChecksumConfigured && !this.useHBaseChecksum
212         && (this.hbaseChecksumOffCount-- < 0)) {
213       // The stream we need is already open (because we were using HBase checksum in the past).
214       assert this.streamNoFsChecksum != null;
215       this.useHBaseChecksum = true;
216     }
217   }
218 
219   /** Close stream(s) if necessary. */
220   public void close() throws IOException {
221     if (!doCloseStreams) return;
222     try {
223       if (stream != streamNoFsChecksum && streamNoFsChecksum != null) {
224         streamNoFsChecksum.close();
225         streamNoFsChecksum = null;
226       }
227     } finally {
228       if (stream != null) {
229         stream.close();
230         stream = null;
231       }
232     }
233   }
234 
235   public HFileSystem getHfs() {
236     return this.hfs;
237   }
238 }