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

Vinayakumar B commented on HADOOP-12502:
----------------------------------------

bq. Doesn't ChecksumFileSystem override listStatus() to supply it's own, 
different DEFAULT_FILTER?
Yes, it overrides. But {{ChecksumFileSystem#listStatus()}} will not be called 
without above change. Here's why,
*ChecksumFileSystem* extends *FilterFileSystem* and *FilterFileSystem* 
overrides {{listStatusIterator(Path p)}} and calls {{fs.listStatusIterator()}} 
internally, so the actuall fs used to call {{listStatusIterator(..)}} is not 
the ChecksumFileSystem, but the underlying FileSystem. i.e. RawLocalFileSystem 
in this case.
  {code:java}
  /** Return a remote iterator for listing in a directory */
   @Override
  public RemoteIterator<FileStatus> listStatusIterator(Path f)
    throws IOException {
     return fs.listStatusIterator(f);
   }
{code}
So by overriding {{listStatusIterarator()}} in checksum file system, 
{{ChecksumFileSystem#DEFAULT_FILTER }} can be applied.

bq. It seems like FileSystem#listStatusBatch() would call 
ChecksumFileSystem#listStatus() which does specify a filter that excludes the 
crc files?
Thats interesting implementation, I missed to see that. As I said above, still 
changes in ChecksumFileSystem required. But  {{FileSystem#listStatusBatch()}} 
 make changes in the {{ChecksumFileSystem#listStatusIterator(..)}} to oneliner.
Will upload the updated patch soon.

bq. Also, can you comment on the testing you've done so far on the latest 
patch? Thanks!
*  With this approach (not the exact patch), we have been running the 
production clusters for more than 2 years. Did not see any side effects. So I 
think its good to go.
*  As I said, this approach benefits only FileSystems with true ability of 
RemoteIterators (HDFS for now) it will not change anything else for other 
filesystems.
*  And also, even for HDFS, for 'ls' if sorting is required no memory 
optimizations will be possible as complete set of results to be obtained before 
sorting, so iterator approach is not used in FsShell (still HDFS internally 
uses).


> SetReplication OutOfMemoryError
> -------------------------------
>
>                 Key: HADOOP-12502
>                 URL: https://issues.apache.org/jira/browse/HADOOP-12502
>             Project: Hadoop Common
>          Issue Type: Bug
>    Affects Versions: 2.3.0
>            Reporter: Philipp Schuegerl
>            Assignee: Vinayakumar B
>            Priority: Major
>         Attachments: HADOOP-12502-01.patch, HADOOP-12502-02.patch, 
> HADOOP-12502-03.patch, HADOOP-12502-04.patch, HADOOP-12502-05.patch, 
> HADOOP-12502-06.patch, HADOOP-12502-07.patch, HADOOP-12502-08.patch, 
> HADOOP-12502-09.patch
>
>
> Setting the replication of a HDFS folder recursively can run out of memory. 
> E.g. with a large /var/log directory:
> hdfs dfs -setrep -R -w 1 /var/log
> Exception in thread "main" java.lang.OutOfMemoryError: GC overhead limit 
> exceeded
>       at java.util.Arrays.copyOfRange(Arrays.java:2694)
>       at java.lang.String.<init>(String.java:203)
>       at java.lang.String.substring(String.java:1913)
>       at java.net.URI$Parser.substring(URI.java:2850)
>       at java.net.URI$Parser.parse(URI.java:3046)
>       at java.net.URI.<init>(URI.java:753)
>       at org.apache.hadoop.fs.Path.initialize(Path.java:203)
>       at org.apache.hadoop.fs.Path.<init>(Path.java:116)
>       at org.apache.hadoop.fs.Path.<init>(Path.java:94)
>       at 
> org.apache.hadoop.hdfs.protocol.HdfsFileStatus.getFullPath(HdfsFileStatus.java:222)
>       at 
> org.apache.hadoop.hdfs.protocol.HdfsFileStatus.makeQualified(HdfsFileStatus.java:246)
>       at 
> org.apache.hadoop.hdfs.DistributedFileSystem.listStatusInternal(DistributedFileSystem.java:689)
>       at 
> org.apache.hadoop.hdfs.DistributedFileSystem.access$600(DistributedFileSystem.java:102)
>       at 
> org.apache.hadoop.hdfs.DistributedFileSystem$14.doCall(DistributedFileSystem.java:712)
>       at 
> org.apache.hadoop.hdfs.DistributedFileSystem$14.doCall(DistributedFileSystem.java:708)
>       at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>       at 
> org.apache.hadoop.hdfs.DistributedFileSystem.listStatus(DistributedFileSystem.java:708)
>       at 
> org.apache.hadoop.fs.shell.PathData.getDirectoryContents(PathData.java:268)
>       at org.apache.hadoop.fs.shell.Command.recursePath(Command.java:347)
>       at org.apache.hadoop.fs.shell.Command.processPaths(Command.java:308)
>       at org.apache.hadoop.fs.shell.Command.recursePath(Command.java:347)
>       at org.apache.hadoop.fs.shell.Command.processPaths(Command.java:308)
>       at org.apache.hadoop.fs.shell.Command.recursePath(Command.java:347)
>       at org.apache.hadoop.fs.shell.Command.processPaths(Command.java:308)
>       at org.apache.hadoop.fs.shell.Command.recursePath(Command.java:347)
>       at org.apache.hadoop.fs.shell.Command.processPaths(Command.java:308)
>       at org.apache.hadoop.fs.shell.Command.recursePath(Command.java:347)
>       at org.apache.hadoop.fs.shell.Command.processPaths(Command.java:308)
>       at 
> org.apache.hadoop.fs.shell.Command.processPathArgument(Command.java:278)
>       at org.apache.hadoop.fs.shell.Command.processArgument(Command.java:260)
>       at org.apache.hadoop.fs.shell.Command.processArguments(Command.java:244)
>       at 
> org.apache.hadoop.fs.shell.SetReplication.processArguments(SetReplication.java:76)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to