You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2020/04/01 21:42:03 UTC
[hbase] branch branch-2.3 updated: HBASE-24051 Allows indirect
inheritance to CanUnbuffer (#1406)
This is an automated email from the ASF dual-hosted git repository.
stack pushed a commit to branch branch-2.3
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/branch-2.3 by this push:
new 34aca94 HBASE-24051 Allows indirect inheritance to CanUnbuffer (#1406)
34aca94 is described below
commit 34aca9457387023128b3d3879794b9cf0808eb7b
Author: 申胜利 <48...@users.noreply.github.com>
AuthorDate: Thu Apr 2 05:39:00 2020 +0800
HBASE-24051 Allows indirect inheritance to CanUnbuffer (#1406)
Signed-off-by: stack <st...@apache.org>
---
.../hadoop/hbase/io/FSDataInputStreamWrapper.java | 25 +++---
.../hbase/io/TestFSDataInputStreamWrapper.java | 88 ++++++++++++++++++++++
2 files changed, 99 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 989d0aa..9562f99 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 0000000..9b031af
--- /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<? 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;
+ }
+ }
+}