blob: 3176fb92890edd1933c210b53fc180ff0e341b97 [file] [log] [blame]
/*
* 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.wal;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.net.URI;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.util.Progressable;
/**
* Create a non-abstract "proxy" for FileSystem because FileSystem is an abstract class and not an
* interface. Only interfaces can be used with the Java Proxy class to override functionality via an
* InvocationHandler.
*/
public class FileSystemProxy extends FileSystem {
private final FileSystem real;
public FileSystemProxy(FileSystem real) {
this.real = real;
}
@Override
public FSDataInputStream open(Path p) throws IOException {
return real.open(p);
}
@Override
public URI getUri() {
return real.getUri();
}
@Override
public FSDataInputStream open(Path f, int bufferSize) throws IOException {
return real.open(f, bufferSize);
}
@Override
public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite,
int bufferSize, short replication, long blockSize, Progressable progress) throws IOException {
return real.create(f, permission, overwrite, bufferSize, replication, blockSize, progress);
}
@Override
public FSDataOutputStream append(Path f, int bufferSize, Progressable progress)
throws IOException {
return real.append(f, bufferSize, progress);
}
@Override
public boolean rename(Path src, Path dst) throws IOException {
return real.rename(src, dst);
}
@Override
public boolean delete(Path f, boolean recursive) throws IOException {
return real.delete(f, recursive);
}
@Override
public FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException {
return real.listStatus(f);
}
@Override
public void setWorkingDirectory(Path new_dir) {
real.setWorkingDirectory(new_dir);
}
@Override
public Path getWorkingDirectory() {
return real.getWorkingDirectory();
}
@Override
public boolean mkdirs(Path f, FsPermission permission) throws IOException {
return real.mkdirs(f, permission);
}
@Override
public FileStatus getFileStatus(Path f) throws IOException {
return real.getFileStatus(f);
}
}