From 17586aa1608116bbd70b25ea43e0208fb2da4d0f 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:43:28 +0800 Subject: [PATCH 1/6] 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 extends InputStream> 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 dc16ed1a08fb35c8b7cc2da1c4d1682b05dee9d4 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 14:36:43 +0800 Subject: [PATCH 2/6] Modify the UT --- .../io/TestFSDataInputStreamWrapper.java | 30 ++++++++++++++++++- 1 file changed, 29 insertions(+), 1 deletion(-) 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..9b031afee8d1 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,18 +1,46 @@ +/** + * 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 unbuffer() { + public void TestUnbuffer() { InputStream wrappedStream = new SonStream(); final Class extends InputStream> streamClass = wrappedStream.getClass(); Class>[] streamInterfaces = streamClass.getInterfaces(); From 561a0e2ae5dab365052937a9ed416022f0f95bc1 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 14:40:02 +0800 Subject: [PATCH 3/6] 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 7d64ef0b4e85655ef02231209bf7d02b80b58e29 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 15:08:46 +0800 Subject: [PATCH 4/6] Modify the UT --- .../io/TestFSDataInputStreamWrapper.java | 65 ++++++++----------- 1 file changed, 27 insertions(+), 38 deletions(-) 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 9b031afee8d1..8b3c988fdbf5 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 @@ -17,72 +17,61 @@ */ package org.apache.hadoop.hbase.io; -import org.apache.hadoop.fs.CanUnbuffer; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseClassTestRule; +import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.testclassification.IOTests; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.junit.Assert; +import org.junit.BeforeClass; 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; +import java.nio.ByteBuffer; @Category({IOTests.class, SmallTests.class}) public class TestFSDataInputStreamWrapper { - private Method unbuffer = null; + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static FileSystem fs; @ClassRule public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestFSDataInputStreamWrapper.class); + @BeforeClass + public static void setUp() throws Exception { + fs = TEST_UTIL.getTestFileSystem(); + } @Test - public void TestUnbuffer() { - InputStream wrappedStream = new SonStream(); - final Class extends InputStream> 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 void TestUnbuffer() throws IOException { + SonStream stream = new SonStream(fs.open(new Path("/p"))); + FSDataInputStreamWrapper fsDISW = new FSDataInputStreamWrapper(stream); + fsDISW.unbuffer(); + Assert.assertEquals(stream.buffer, null); } - public class SonStream extends FatherStream { - @Override - public void unbuffer() { + public class SonStream extends FSDataInputStream { - } - } + public ByteBuffer buffer = ByteBuffer.allocate(1024); - public class FatherStream extends InputStream implements CanUnbuffer { - - @Override - public void unbuffer() { + public SonStream(InputStream in) { + super(in); + } + public InputStream getWrappedStream() { + return this; } @Override - public int read() throws IOException { - return 0; + public void unbuffer() { + buffer = null; } } + } From 80fccb18242fca39824b35ff5ee79ec001fe446f 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 16:58:03 +0800 Subject: [PATCH 5/6] Modify the UT --- .../io/TestFSDataInputStreamWrapper.java | 27 ++++++++++++++++--- 1 file changed, 23 insertions(+), 4 deletions(-) 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 8b3c988fdbf5..1d8255594bf2 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 @@ -6,9 +6,9 @@ * 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 - * + *
+ * 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. @@ -30,6 +30,7 @@ import org.junit.Test; import org.junit.experimental.categories.Category; +import java.io.File; import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; @@ -50,7 +51,10 @@ public static void setUp() throws Exception { @Test public void TestUnbuffer() throws IOException { - SonStream stream = new SonStream(fs.open(new Path("/p"))); + String testDir = "/tmp/"; + String testFile = "test.txt"; + createFile(testDir, testFile); + SonStream stream = new SonStream(fs.open(new Path(testDir + testFile))); FSDataInputStreamWrapper fsDISW = new FSDataInputStreamWrapper(stream); fsDISW.unbuffer(); Assert.assertEquals(stream.buffer, null); @@ -74,4 +78,19 @@ public void unbuffer() { } } + public static void createFile(String directory, String filename) { + File file = new File(directory); + if (!file.exists()) { + file.mkdirs(); + } + File file2 = new File(directory, filename); + if (!file2.exists()) { + try { + file2.createNewFile(); + } catch (IOException e) { + e.printStackTrace(); + } + } + } + } From fbf8a51af67c8356e294817505b449d18efdf015 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 17:00:37 +0800 Subject: [PATCH 6/6] Modify the UT --- .../hadoop/hbase/io/TestFSDataInputStreamWrapper.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 1d8255594bf2..155a71340ad0 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 @@ -6,9 +6,9 @@ * 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 - *
+ * + * 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.