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

Nico Kruber edited comment on FLINK-8336 at 2/27/18 10:59 AM:
--------------------------------------------------------------

According to 
https://docs.aws.amazon.com/AmazonS3/latest/dev/Introduction.html#ConsistencyModel:
{quote}
Amazon S3 provides read-after-write consistency for PUTS of new objects in your 
S3 bucket in all regions with one caveat. The caveat is that if you make a HEAD 
or GET request to the key name (to find if the object exists) before creating 
the object, Amazon S3 provides eventual consistency for read-after-write.
{quote}
However, depending on the Hadoop version that is being used, there may be an 
existence check before writing the file even though we intend to overwrite it, 
e.g. in 2.4.1 the following is called:
{code}
org.apache.flink.yarn.Utils#setupLocalResource(...)
-> org.apache.hadoop.fs.s3.S3FileSystem#copyFromLocalFile(...) inherits from 
org.apache.hadoop.fs.FileSystem#copyFromLocalFile(...)
-> org.apache.hadoop.fs.FileUtil#copy(...)
-> org.apache.hadoop.fs.FileUtil#checkDest
-> executes:
    if (dstFS.exists(dst)) {
      FileStatus sdst = dstFS.getFileStatus(dst);
//...
{code}
and this leads to the eventually consistent read after writing the resource to 
S3.

S3AFileSystem from Hadoop 2.8.1 is better here:
{code}
org.apache.hadoop.fs.s3a.S3AFileSystem#copyFromLocalFile(...)
-> org.apache.hadoop.fs.s3a.S3AFileSystem#innerCopyFromLocalFile(...)
-> executes:
    if (!overwrite && exists(dst)) {
      throw new FileAlreadyExistsException(dst + " already exists");
    }
{code}

Our shaded flink-s3-fs-hadoop uses Hadoop 2.8.1 and should be safe. However, 
flink-s3-fs-presto is based on {{FileSystem#copyFromLocalFile(...)}} from 
Hadoop 2.7.3 and this code (even in Hadoop 2.8.1, for that matter) always 
executes the existence check mentioned above. However, presto's S3 
implementation of {{getFileStatus}} does perform retries with exponential 
backoff which probably solves the issue for us.


was (Author: nicok):
According to 
https://docs.aws.amazon.com/AmazonS3/latest/dev/Introduction.html#ConsistencyModel:
{quote}
Amazon S3 provides read-after-write consistency for PUTS of new objects in your 
S3 bucket in all regions with one caveat. The caveat is that if you make a HEAD 
or GET request to the key name (to find if the object exists) before creating 
the object, Amazon S3 provides eventual consistency for read-after-write.
{quote}
However, depending on the Hadoop version that is being used, there may be an 
existence check before writing the file even though we intend to overwrite it, 
e.g. in 2.4.1 the following is called:
{code}
org.apache.flink.yarn.Utils#setupLocalResource(...)
-> org.apache.hadoop.fs.s3.S3FileSystem#copyFromLocalFile(...) inherits from 
org.apache.hadoop.fs.FileSystem#copyFromLocalFile(...)
-> org.apache.hadoop.fs.FileUtil#copy(...)
-> org.apache.hadoop.fs.FileUtil#checkDest
-> executes:
    if (dstFS.exists(dst)) {
      FileStatus sdst = dstFS.getFileStatus(dst);
//...
{code}
and this leads to the eventually consistent read after writing the resource to 
S3.

S3AFileSystem from Hadoop 2.8.1 is better here:
{code}
org.apache.hadoop.fs.s3a.S3AFileSystem#copyFromLocalFile(...)
-> org.apache.hadoop.fs.s3a.S3AFileSystem#innerCopyFromLocalFile(...)
-> executes:
    if (!overwrite && exists(dst)) {
      throw new FileAlreadyExistsException(dst + " already exists");
    }
{code}

Our shaded flink-s3-fs-hadoop uses Hadoop 2.8.1 and should be safe. However, 
flink-s3-fs-presto is based on {{FileSystem#copyFromLocalFile(...)}} from 
Hadoop 2.7.3 and this code (even in Hadoop 2.8.1, for that matter) always 
executes the existence check mentioned above.

> YarnFileStageTestS3ITCase.testRecursiveUploadForYarnS3 test instability
> -----------------------------------------------------------------------
>
>                 Key: FLINK-8336
>                 URL: https://issues.apache.org/jira/browse/FLINK-8336
>             Project: Flink
>          Issue Type: Bug
>          Components: FileSystem, Tests, YARN
>    Affects Versions: 1.5.0
>            Reporter: Till Rohrmann
>            Priority: Blocker
>              Labels: test-stability
>             Fix For: 1.5.0, 1.4.2
>
>
> The {{YarnFileStageTestS3ITCase.testRecursiveUploadForYarnS3}} fails on 
> Travis. I suspect that this has something to do with the consistency 
> guarantees S3 gives us.
> https://travis-ci.org/tillrohrmann/flink/jobs/323930297



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

Reply via email to