Skip to content

Commit 554678e

Browse files
committed
Revert "HBASE-24051 Allows indirect inheritance to CanUnbuffer (#1406)"
This reverts commit ae1f928.
1 parent ae1f928 commit 554678e

File tree

2 files changed

+14
-99
lines changed

2 files changed

+14
-99
lines changed

hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java

Lines changed: 14 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,6 @@
2525
import java.util.concurrent.atomic.AtomicInteger;
2626

2727
import org.apache.commons.io.IOUtils;
28-
import org.apache.hadoop.fs.CanUnbuffer;
2928
import org.apache.hadoop.fs.FSDataInputStream;
3029
import org.apache.hadoop.fs.FileSystem;
3130
import org.apache.hadoop.fs.Path;
@@ -332,18 +331,22 @@ public void unbuffer() {
332331
if (this.instanceOfCanUnbuffer == null) {
333332
// To ensure we compute whether the stream is instance of CanUnbuffer only once.
334333
this.instanceOfCanUnbuffer = false;
335-
if(wrappedStream instanceof CanUnbuffer){
336-
try {
337-
this.unbuffer = streamClass.getDeclaredMethod("unbuffer");
338-
} catch (NoSuchMethodException | SecurityException e) {
339-
if (isLogTraceEnabled) {
340-
LOG.trace("Failed to find 'unbuffer' method in class " + streamClass
341-
+ " . So there may be a TCP socket connection "
342-
+ "left open in CLOSE_WAIT state.", e);
334+
Class<?>[] streamInterfaces = streamClass.getInterfaces();
335+
for (Class c : streamInterfaces) {
336+
if (c.getCanonicalName().toString().equals("org.apache.hadoop.fs.CanUnbuffer")) {
337+
try {
338+
this.unbuffer = streamClass.getDeclaredMethod("unbuffer");
339+
} catch (NoSuchMethodException | SecurityException e) {
340+
if (isLogTraceEnabled) {
341+
LOG.trace("Failed to find 'unbuffer' method in class " + streamClass
342+
+ " . So there may be a TCP socket connection "
343+
+ "left open in CLOSE_WAIT state.", e);
344+
}
345+
return;
343346
}
344-
return;
347+
this.instanceOfCanUnbuffer = true;
348+
break;
345349
}
346-
this.instanceOfCanUnbuffer = true;
347350
}
348351
}
349352
if (this.instanceOfCanUnbuffer) {

hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestFSDataInputStreamWrapper.java

Lines changed: 0 additions & 88 deletions
This file was deleted.

0 commit comments

Comments
 (0)