hadoop-common-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Manjunath Anand (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HADOOP-13870) Incorrect behavior of copyFromLocalFile on implementations of FilterFileSystem
Date Thu, 19 Jan 2017 06:59:26 GMT

    [ https://issues.apache.org/jira/browse/HADOOP-13870?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15829422#comment-15829422
] 

Manjunath Anand commented on HADOOP-13870:
------------------------------------------

Sorry for looking late into this.

This is not an issue and it works as expected. The reason why test_incorrect doesnt fail is
that the copyFromLocalFile call done internally calls the create method of ChecksumFileSystem
which overrides the create method implemented in the FilterFileSystem.

Whereas the wrappedFs.create(OUT_PATH) in the test_correct fails because its a direct call
made to FilterFileSystem and not to the LocalFileSystem which extends the ChecksumFileSystem.


If the FailingFileSystem is made to extend the LocalFileSystem instead of FilterFileSystem
then you will see both methods failing.

The below stacktrace says it all:-

{code}

java.lang.Thread.State: RUNNABLE
	  at org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:464)
	  at org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:443)
	  at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1073)
	  at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1053)
	  at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:942)
	  at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:391)
	  at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:364)
	  at org.apache.hadoop.fs.FileUtil.copy(FileUtil.java:314)
	  at org.apache.hadoop.fs.LocalFileSystem.copyFromLocalFile(LocalFileSystem.java:82)
	  at org.apache.hadoop.fs.FilterFileSystem.copyFromLocalFile(FilterFileSystem.java:332)
	  at org.apache.hadoop.fs.FileSystem.copyFromLocalFile(FileSystem.java:2241)
	  at org.apache.hadoop.util.CopyFromLocalFileTest.test_incorrect(CopyFromLocalFileTest.java:48)

java.lang.Thread.State: RUNNABLE
	  at org.apache.hadoop.util.CopyFromLocalFileTest$FailingFileSystem.create(CopyFromLocalFileTest.java:60)
	  at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1073)
	  at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1053)
	  at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:942)
	  at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:930)
	  at org.apache.hadoop.util.CopyFromLocalFileTest.test_correct(CopyFromLocalFileTest.java:42)

{code}

> 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); // this call fails
>         IOUtils.copy(in, out);
>     }
>     @Test
>     public void test_incorrect() throws IOException {
>         wrappedFs.copyFromLocalFile(DATA_PATH, OUT_PATH); // this call does not fail
>     }
>     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-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org


Mime
View raw message