From 30f5852fc2d9743e312b335cf278918946bdd1b9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=94=B3=E8=83=9C=E5=88=A9?= <48829688+shenshengli@users.noreply.github.com> Date: Thu, 2 Apr 2020 05:39:00 +0800 Subject: [PATCH] HBASE-24051 Allows indirect inheritance to CanUnbuffer (#1406) Signed-off-by: stack --- .../hbase/io/FSDataInputStreamWrapper.java | 25 +++--- .../io/TestFSDataInputStreamWrapper.java | 88 +++++++++++++++++++ 2 files changed, 99 insertions(+), 14 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestFSDataInputStreamWrapper.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java index 989d0aab2e6..9562f99e645 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FSDataInputStreamWrapper.java @@ -25,6 +25,7 @@ import java.lang.reflect.Method; import java.util.concurrent.atomic.AtomicInteger; import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.CanUnbuffer; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -331,22 +332,18 @@ public class FSDataInputStreamWrapper implements Closeable { if (this.instanceOfCanUnbuffer == null) { // To ensure we compute whether the stream is instance of CanUnbuffer only once. this.instanceOfCanUnbuffer = false; - Class[] streamInterfaces = streamClass.getInterfaces(); - for (Class c : streamInterfaces) { - if (c.getCanonicalName().toString().equals("org.apache.hadoop.fs.CanUnbuffer")) { - try { - this.unbuffer = streamClass.getDeclaredMethod("unbuffer"); - } catch (NoSuchMethodException | SecurityException e) { - if (isLogTraceEnabled) { - LOG.trace("Failed to find 'unbuffer' method in class " + streamClass - + " . So there may be a TCP socket connection " - + "left open in CLOSE_WAIT state.", e); - } - return; + if(wrappedStream instanceof CanUnbuffer){ + try { + this.unbuffer = streamClass.getDeclaredMethod("unbuffer"); + } catch (NoSuchMethodException | SecurityException e) { + if (isLogTraceEnabled) { + LOG.trace("Failed to find 'unbuffer' method in class " + streamClass + + " . So there may be a TCP socket connection " + + "left open in CLOSE_WAIT state.", e); } - this.instanceOfCanUnbuffer = true; - break; + return; } + this.instanceOfCanUnbuffer = true; } } if (this.instanceOfCanUnbuffer) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestFSDataInputStreamWrapper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestFSDataInputStreamWrapper.java new file mode 100644 index 00000000000..9b031afee8d --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestFSDataInputStreamWrapper.java @@ -0,0 +1,88 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hbase.io; + +import org.apache.hadoop.fs.CanUnbuffer; +import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.testclassification.IOTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.io.IOException; +import java.io.InputStream; +import java.lang.reflect.Method; + +@Category({IOTests.class, SmallTests.class}) +public class TestFSDataInputStreamWrapper { + private Method unbuffer = null; + + @ClassRule + public static final HBaseClassTestRule CLASS_RULE = + HBaseClassTestRule.forClass(TestFSDataInputStreamWrapper.class); + + + @Test + public void TestUnbuffer() { + InputStream wrappedStream = new SonStream(); + final Class streamClass = wrappedStream.getClass(); + Class[] streamInterfaces = streamClass.getInterfaces(); + for (Class c : streamInterfaces) { + if (c.getCanonicalName().toString().equals("org.apache.hadoop.fs.CanUnbuffer")) { + try { + this.unbuffer = streamClass.getDeclaredMethod("unbuffer"); + } catch (NoSuchMethodException | SecurityException e) { + return; + } + break; + } + } + Assert.assertEquals(false, unbuffer != null); + unbuffer = null; + if (wrappedStream instanceof CanUnbuffer) { + try { + this.unbuffer = streamClass.getDeclaredMethod("unbuffer"); + } catch (NoSuchMethodException | SecurityException e) { + return; + } + } + Assert.assertEquals(true, unbuffer != null); + } + + public class SonStream extends FatherStream { + @Override + public void unbuffer() { + + } + } + + public class FatherStream extends InputStream implements CanUnbuffer { + + @Override + public void unbuffer() { + + } + + @Override + public int read() throws IOException { + return 0; + } + } +}