Joe Ellis created HADOOP-13870: ---------------------------------- Summary: Incorrect behavior of copyFromLocalFile on implementations of FilterFileSystem Key: HADOOP-13870 URL: https://issues.apache.org/jira/browse/HADOOP-13870 Project: Hadoop Common Issue Type: Bug Components: fs Reporter: Joe Ellis
This may be an incorrect assumption on my part, but it was my belief that overriding the create method on a FilterFileSystem was sufficient to intercept all calls that would write data through the FileSystem. This is apparently not true because calling copyFromLocalFile on the FilterFileSystem eventually invokes the create method on the wrapped FileSystem. I would expect open -> create -> copy(opened, created) to be functionally equivalent to copyFromLocal when using the same input and output paths, but this is not the case. {code:java} import java.io.IOException; import java.net.URI; import java.nio.charset.StandardCharsets; import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FilterFileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.util.Progressable; import org.junit.Before; import org.junit.Test; public final class CopyFromLocalFileTest { private static final Path DATA_PATH = new Path("file:///tmp/test_in"); private static final Path OUT_PATH = new Path("file:///tmp/test_out"); private FileSystem localFs; private FileSystem wrappedFs; @Before public void before() throws IOException { localFs = FileSystem.get(URI.create("file:///"), new Configuration()); wrappedFs = new FailingFileSystem(localFs); FSDataOutputStream tmpFile = localFs.create(DATA_PATH); byte[] bytes = "data".getBytes(StandardCharsets.UTF_8); tmpFile.write(bytes); tmpFile.close(); } @Test public void test_correct() throws IOException { FSDataInputStream in = wrappedFs.open(DATA_PATH); FSDataOutputStream out = wrappedFs.create(OUT_PATH); IOUtils.copy(in, out); } @Test public void test_incorrect() throws IOException { wrappedFs.copyFromLocalFile(DATA_PATH, OUT_PATH); } private static final class FailingFileSystem extends FilterFileSystem { public FailingFileSystem(FileSystem fs) { super(fs); } @Override public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { throw new IOException("fail"); } } } {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: common-dev-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-dev-h...@hadoop.apache.org