1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
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
37
38
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
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70 private volatile FSDataInputStream stream = null;
71 private volatile FSDataInputStream streamNoFsChecksum = null;
72 private final Object streamNoFsChecksumFirstCreateLock = new Object();
73
74
75 private boolean useHBaseChecksumConfigured;
76
77
78
79
80
81 private volatile boolean useHBaseChecksum;
82
83
84
85 private volatile int hbaseChecksumOffCount = -1;
86
87 private Boolean instanceOfCanUnbuffer = null;
88
89
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
124
125
126 this.hfs = (fs instanceof HFileSystem) ? (HFileSystem)fs : new HFileSystem(fs);
127
128
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
135 }
136 }
137
138
139
140
141
142
143
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
156 this.stream.close();
157 this.stream = null;
158 }
159 }
160
161
162 public FSDataInputStreamWrapper(FSDataInputStream fsdis) {
163 this(fsdis, fsdis);
164 }
165
166
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
179
180 public boolean shouldUseHBaseChecksum() {
181 return this.useHBaseChecksum;
182 }
183
184
185
186
187
188
189 public FSDataInputStream getStream(boolean useHBaseChecksum) {
190 return useHBaseChecksum ? this.streamNoFsChecksum : this.stream;
191 }
192
193
194
195
196
197 public FSDataInputStream fallbackToFsChecksum(int offCount) throws IOException {
198
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
216 public void checksumOk() {
217 if (this.useHBaseChecksumConfigured && !this.useHBaseChecksum
218 && (this.hbaseChecksumOffCount-- < 0)) {
219
220 assert this.streamNoFsChecksum != null;
221 this.useHBaseChecksum = true;
222 }
223 }
224
225 private void updateInputStreamStatistics(FSDataInputStream stream) {
226
227 if (stream instanceof HdfsDataInputStream) {
228
229
230
231
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
272 public void close() {
273 if (!doCloseStreams) {
274 return;
275 }
276 updateInputStreamStatistics(this.streamNoFsChecksum);
277
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
289
290
291
292
293
294
295
296
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
304
305
306 final Class<? extends InputStream> streamClass = wrappedStream.getClass();
307 if (this.instanceOfCanUnbuffer == null) {
308
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 }