|
19 | 19 |
|
20 | 20 | import java.io.Closeable; |
21 | 21 | import java.io.IOException; |
22 | | -import java.io.InputStream; |
23 | 22 | import java.util.concurrent.atomic.AtomicInteger; |
24 | | -import org.apache.hadoop.fs.CanUnbuffer; |
25 | 23 | import org.apache.hadoop.fs.FSDataInputStream; |
26 | 24 | import org.apache.hadoop.fs.FileSystem; |
27 | 25 | import org.apache.hadoop.fs.Path; |
28 | 26 | import org.apache.hadoop.hbase.fs.HFileSystem; |
29 | 27 | import org.apache.hadoop.hdfs.client.HdfsDataInputStream; |
30 | 28 | import org.apache.yetus.audience.InterfaceAudience; |
31 | | -import org.slf4j.Logger; |
32 | | -import org.slf4j.LoggerFactory; |
33 | 29 |
|
34 | 30 | import org.apache.hbase.thirdparty.com.google.common.io.Closeables; |
35 | 31 |
|
|
40 | 36 | */ |
41 | 37 | @InterfaceAudience.Private |
42 | 38 | public class FSDataInputStreamWrapper implements Closeable { |
43 | | - private static final Logger LOG = LoggerFactory.getLogger(FSDataInputStreamWrapper.class); |
44 | | - private static final boolean isLogTraceEnabled = LOG.isTraceEnabled(); |
45 | 39 |
|
46 | 40 | private final HFileSystem hfs; |
47 | 41 | private final Path path; |
@@ -94,9 +88,6 @@ private static class ReadStatistics { |
94 | 88 | long totalZeroCopyBytesRead; |
95 | 89 | } |
96 | 90 |
|
97 | | - private Boolean instanceOfCanUnbuffer = null; |
98 | | - private CanUnbuffer unbuffer = null; |
99 | | - |
100 | 91 | protected Path readerPath; |
101 | 92 |
|
102 | 93 | public FSDataInputStreamWrapper(FileSystem fs, Path path) throws IOException { |
@@ -314,41 +305,22 @@ public HFileSystem getHfs() { |
314 | 305 | * stream, the current socket will be closed and a new socket will be opened to serve the |
315 | 306 | * requests. |
316 | 307 | */ |
317 | | - @SuppressWarnings({ "rawtypes" }) |
318 | 308 | public void unbuffer() { |
| 309 | + // todo: it may make sense to always unbuffer both streams. we'd need to carefully |
| 310 | + // research the usages to know if that is safe. for now just do the current. |
319 | 311 | FSDataInputStream stream = this.getStream(this.shouldUseHBaseChecksum()); |
320 | 312 | if (stream != null) { |
321 | | - InputStream wrappedStream = stream.getWrappedStream(); |
322 | | - // CanUnbuffer interface was added as part of HDFS-7694 and the fix is available in Hadoop |
323 | | - // 2.6.4+ and 2.7.1+ versions only so check whether the stream object implements the |
324 | | - // CanUnbuffer interface or not and based on that call the unbuffer api. |
325 | | - final Class<? extends InputStream> streamClass = wrappedStream.getClass(); |
326 | | - if (this.instanceOfCanUnbuffer == null) { |
327 | | - // To ensure we compute whether the stream is instance of CanUnbuffer only once. |
328 | | - this.instanceOfCanUnbuffer = false; |
329 | | - if (wrappedStream instanceof CanUnbuffer) { |
330 | | - this.unbuffer = (CanUnbuffer) wrappedStream; |
331 | | - this.instanceOfCanUnbuffer = true; |
332 | | - } |
333 | | - } |
334 | | - if (this.instanceOfCanUnbuffer) { |
335 | | - try { |
336 | | - this.unbuffer.unbuffer(); |
337 | | - } catch (UnsupportedOperationException e) { |
338 | | - if (isLogTraceEnabled) { |
339 | | - LOG.trace("Failed to invoke 'unbuffer' method in class " + streamClass |
340 | | - + " . So there may be the stream does not support unbuffering.", e); |
341 | | - } |
342 | | - } |
343 | | - } else { |
344 | | - if (isLogTraceEnabled) { |
345 | | - LOG.trace("Failed to find 'unbuffer' method in class " + streamClass); |
346 | | - } |
347 | | - } |
| 313 | + stream.unbuffer(); |
348 | 314 | } |
349 | 315 | } |
350 | 316 |
|
351 | 317 | public Path getReaderPath() { |
352 | 318 | return readerPath; |
353 | 319 | } |
| 320 | + |
| 321 | + // For tests |
| 322 | + void setShouldUseHBaseChecksum() { |
| 323 | + useHBaseChecksumConfigured = true; |
| 324 | + useHBaseChecksum = true; |
| 325 | + } |
354 | 326 | } |
0 commit comments