You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by "Steve Loughran (Jira)" <ji...@apache.org> on 2022/03/10 13:56:00 UTC

[jira] [Updated] (HADOOP-18157) S3AFileSystem cannot find a fake dir on minio

     [ https://issues.apache.org/jira/browse/HADOOP-18157?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Steve Loughran updated HADOOP-18157:
------------------------------------
    Summary: S3AFileSystem cannot find a fake dir on minio  (was: S3AFileSystem cannot find a fake dir any more)

> S3AFileSystem cannot find a fake dir on minio
> ---------------------------------------------
>
>                 Key: HADOOP-18157
>                 URL: https://issues.apache.org/jira/browse/HADOOP-18157
>             Project: Hadoop Common
>          Issue Type: Bug
>          Components: fs/s3
>    Affects Versions: 3.2.2, 3.3.1
>         Environment: minio
>            Reporter: JacobZheng
>            Priority: Minor
>
> I am using spark and delta lake for reading and writing datasets. After I upgraded the hadoop version, I found that writing files reported an error, indicating that the folder does not exist.
> {code:java}
>     java.io.FileNotFoundException: No such file or directory: s3a://table_cache/t5ffe773c987844cbba0aafd/_delta_log
>         at org.apache.hadoop.fs.s3a.S3AFileSystem.s3GetFileStatus(S3AFileSystem.java:3356)
>         at org.apache.hadoop.fs.s3a.S3AFileSystem.innerGetFileStatus(S3AFileSystem.java:3185)
>         at org.apache.hadoop.fs.s3a.S3AFileSystem.getFileStatus(S3AFileSystem.java:3053)
>         at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1760)
>         at org.apache.hadoop.fs.s3a.S3AFileSystem.exists(S3AFileSystem.java:4263)
>         at org.apache.spark.sql.delta.storage.S3SingleDriverLogStore.listFromInternal(S3SingleDriverLogStore.scala:120)
>         at org.apache.spark.sql.delta.storage.S3SingleDriverLogStore.exists(S3SingleDriverLogStore.scala:156)
>         at org.apache.spark.sql.delta.storage.S3SingleDriverLogStore.write(S3SingleDriverLogStore.scala:174)
>         at org.apache.spark.sql.delta.OptimisticTransactionImpl.doCommit(OptimisticTransaction.scala:742)
> ...
> {code}
> When this folder has been created, the error is still reported.
> The related commit is 49f8ae96, It delete some code that dealing with Fake directory.
> {code:java}
> // Either a normal file was not found or the probe was skipped.
>       // because the key ended in "/" or it was not in the set of probes.
>       // Look for the dir marker
>       if (probes.contains(StatusProbeEnum.DirMarker)) {
>         String newKey = maybeAddTrailingSlash(key);
>         try {
>           ObjectMetadata meta = getObjectMetadata(newKey);
>           if (objectRepresentsDirectory(newKey, meta.getContentLength())) {
>             LOG.debug("Found file (with /): fake directory");
>             return new S3AFileStatus(Tristate.TRUE, path, username);
>           } else {
>             LOG.warn("Found file (with /): real file? should not happen: {}",
>                 key);
>             return new S3AFileStatus(meta.getContentLength(),
>                     dateToLong(meta.getLastModified()),
>                     path,
>                     getDefaultBlockSize(path),
>                     username,
>                     meta.getETag(),
>                     meta.getVersionId());
>           }
>         } catch (AmazonServiceException e) {
>           if (e.getStatusCode() != SC_404 || isUnknownBucket(e)) {
>             throw translateException("getFileStatus", newKey, e);
>           }
>         } catch (AmazonClientException e) {
>           throw translateException("getFileStatus", newKey, e);
>         }
>       }
>     }
> {code}



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

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org