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

Steve Loughran commented on HADOOP-18019:
-----------------------------------------

ok. 

First, thanks for looking into this in detail. I don't go in the minio myself 
because I have enough to deal with purely on S3 issues and if something works 
on Mineo that doesn't mean it actually works on AWS. I do not have the time.


that "newly created dirs exist" requirement is fundamental, I can imagine 
things failing.

we cut that "necessary?" bit because the list operation which we fall should 
find the dir + / path in the listing. We can go down from a HEAD, HEAD LIST to 
HEAD, LIST, saving money and time. It relies on the LIST call listing 1+ 
objects under a path in the files or prefixes.


Try the same code with logging to debug and see what comes back, especially 
from the AWS/http layers.
{code}
log4j.logger.org.apache.hadoop.fs.s3a=DEBUG
log4j.logger.com.amazonaws.request=DEBUG
log4j.logger.com.amazonaws.thirdparty.apache.http=DEBUG
log4j.logger.org.apache.http=DEBUG
{code}



Also, could you check out hadoop trunk and run the hadoop aws suite against it, 
trying both -Dmarkers=keep and  -Dmarkers=delete? And us know what fails? It's 
a bigger piece of work, but it sets you up for helping to fix it.


> Hadoop 3.3 regression in hadoop/fs/s3a/S3AFileSystem.s3GetFileStatus()
> ----------------------------------------------------------------------
>
>                 Key: HADOOP-18019
>                 URL: https://issues.apache.org/jira/browse/HADOOP-18019
>             Project: Hadoop Common
>          Issue Type: Bug
>          Components: fs/s3
>    Affects Versions: 3.3.0, 3.3.1, 3.3.2
>            Reporter: Ruslan Dautkhanov
>            Priority: Major
>
> Repro code:
> {code:java}
> val conf = new Configuration()  
> conf.set("fs.s3a.endpoint", "http://127.0.0.1:9000";) 
> conf.set("fs.s3a.path.style.access", "true") 
> conf.set("fs.s3a.access.key", "user_access_key") 
> conf.set("fs.s3a.secret.key", "password")  
> val path = new Path("s3a://comcast-test")  
> val fs = path.getFileSystem(conf)  
> fs.mkdirs(new Path("/testdelta/_delta_log"))  
> fs.getFileStatus(new Path("/testdelta/_delta_log")){code}
> Fails with *FileNotFoundException fails* on Minio. The same code works in 
> real S3.
> It also works in Hadoop 3.2 with Minio and earlier versions.
> Only fails on 3.3 and newer Hadoop branches.
> The reason as discovered by [~sadikovi] is actually a more fundamental one - 
> Minio does not have empty directories (sort of), see 
> [https://github.com/minio/minio/issues/2423].
> This works in Hadoop 3.2 because of this infamous "Is this necessary?" block 
> of code
> [https://github.com/apache/hadoop/blob/branch-3.2.0/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java#L2204-L2223]
> that was removed in Hadoop 3.3 -
> [https://github.com/apache/hadoop/blob/branch-3.3.0/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java#L2179]
> and this causes the regression



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

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

Reply via email to