From b31c6374094ae02108fa65cbb4d789e262475959 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: Wed, 1 Apr 2020 10:39:08 +0800 Subject: [PATCH 1/3] HBASE-24051 Allows indirect inheritance to CanUnbuffer --- .../hbase/io/FSDataInputStreamWrapper.java | 25 ++++++++----------- 1 file changed, 11 insertions(+), 14 deletions(-) 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 989d0aab2e67..9562f99e6452 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.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 void unbuffer() { 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) { From 41f8e0d6fe04c36eae1a8aa340c1a5e3ef495b82 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: Wed, 1 Apr 2020 10:47:28 +0800 Subject: [PATCH 2/3] HBASE-24051 Add UT --- .../io/TestFSDataInputStreamWrapper.java | 60 +++++++++++++++++++ 1 file changed, 60 insertions(+) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestFSDataInputStreamWrapper.java 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 000000000000..3bbecd362656 --- /dev/null +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/TestFSDataInputStreamWrapper.java @@ -0,0 +1,60 @@ +package org.apache.hadoop.hbase.io; + +import org.apache.hadoop.fs.CanUnbuffer; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.io.InputStream; +import java.lang.reflect.Method; + +public class TestFSDataInputStreamWrapper { + private Method unbuffer = null; + + @Test + public void unbuffer() { + 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; + } + } +} From 64ab284a86191ad82472535b3721ca48e0683ab9 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: Wed, 1 Apr 2020 11:38:32 +0800 Subject: [PATCH 3/3] Add agreements --- .../hbase/io/TestFSDataInputStreamWrapper.java | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) 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 index 3bbecd362656..ed4ad2a865f2 100644 --- 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 @@ -1,3 +1,20 @@ +/** + * 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;