Skip to content

Commit d4067c5

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

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;
@@ -271,18 +270,22 @@ public void unbuffer() {
271270
if (this.instanceOfCanUnbuffer == null) {
272271
// To ensure we compute whether the stream is instance of CanUnbuffer only once.
273272
this.instanceOfCanUnbuffer = false;
274-
if(wrappedStream instanceof CanUnbuffer){
275-
try {
276-
this.unbuffer = streamClass.getDeclaredMethod("unbuffer");
277-
} catch (NoSuchMethodException | SecurityException e) {
278-
if (isLogTraceEnabled) {
279-
LOG.trace("Failed to find 'unbuffer' method in class " + streamClass
280-
+ " . So there may be a TCP socket connection "
281-
+ "left open in CLOSE_WAIT state.", e);
273+
Class<?>[] streamInterfaces = streamClass.getInterfaces();
274+
for (Class c : streamInterfaces) {
275+
if (c.getCanonicalName().toString().equals("org.apache.hadoop.fs.CanUnbuffer")) {
276+
try {
277+
this.unbuffer = streamClass.getDeclaredMethod("unbuffer");
278+
} catch (NoSuchMethodException | SecurityException e) {
279+
if (isLogTraceEnabled) {
280+
LOG.trace("Failed to find 'unbuffer' method in class " + streamClass
281+
+ " . So there may be a TCP socket connection "
282+
+ "left open in CLOSE_WAIT state.", e);
283+
}
284+
return;
282285
}
283-
return;
286+
this.instanceOfCanUnbuffer = true;
287+
break;
284288
}
285-
this.instanceOfCanUnbuffer = true;
286289
}
287290
}
288291
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)