Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why doesn't the old way work? Is it not finding the unbuffer method via reflection? Thanks.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The old method can only get those that implement Canbuffer directly.

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) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/**
* 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.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;
}
}
}