HBASE-24051 Allows indirect inheritance to CanUnbuffer (#1406)
Signed-off-by: stack <stack@apache.org>
This commit is contained in:
parent
040a0a7ac5
commit
ae1f928b13
|
@ -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,9 +332,7 @@ 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")) {
|
||||
if(wrappedStream instanceof CanUnbuffer){
|
||||
try {
|
||||
this.unbuffer = streamClass.getDeclaredMethod("unbuffer");
|
||||
} catch (NoSuchMethodException | SecurityException e) {
|
||||
|
@ -345,8 +344,6 @@ public class FSDataInputStreamWrapper implements Closeable {
|
|||
return;
|
||||
}
|
||||
this.instanceOfCanUnbuffer = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (this.instanceOfCanUnbuffer) {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue