HBASE-7635 Proxy created by HFileSystem#createReorderingProxy() should implement Closeable (Ted Yu)

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1436594 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Zhihong Yu 2013-01-21 20:29:34 +00:00
parent 4580a66bbb
commit e5ef0c2a04

View File

@ -33,13 +33,12 @@ import java.net.URI;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FilterFileSystem; import org.apache.hadoop.fs.FilterFileSystem;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.LocalFileSystem; import org.apache.hadoop.fs.LocalFileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.DistributedFileSystem;
@ -47,8 +46,9 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks; import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.io.Closeable;
import org.apache.hadoop.util.Progressable; import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.ReflectionUtils;
/** /**
* An encapsulation for the FileSystem object that hbase uses to access * An encapsulation for the FileSystem object that hbase uses to access
@ -259,7 +259,7 @@ public class HFileSystem extends FilterFileSystem {
final ReorderBlocks lrb, final Configuration conf) { final ReorderBlocks lrb, final Configuration conf) {
return (ClientProtocol) Proxy.newProxyInstance return (ClientProtocol) Proxy.newProxyInstance
(cp.getClass().getClassLoader(), (cp.getClass().getClassLoader(),
new Class[]{ClientProtocol.class}, new Class[]{ClientProtocol.class, Closeable.class},
new InvocationHandler() { new InvocationHandler() {
public Object invoke(Object proxy, Method method, public Object invoke(Object proxy, Method method,
Object[] args) throws Throwable { Object[] args) throws Throwable {