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

ASF GitHub Bot commented on FLINK-8402:
---------------------------------------

Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/5624
  
    Okay, the solution is actually in the description of another pull request:
    
    > 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."**
    
https://docs.aws.amazon.com/AmazonS3/latest/dev/Introduction.html#ConsistencyModel
    
    That means that through the `flink-s3-fs-hadoop` connector, we can never 
assume any form of strongly consistent operation, not even the basic 
read-after-write for exact keys. That's a bummer :-(
    
    On the upside, the `flink-s3-fs-presto` does not seem to do that, so we 
might be able to rely on strong consistency for read-after-write.
    
    My original suggestion boiled down to changing the test code as follows, 
which should work if GET is consistent after PUT:
    ```java
    public static boolean pathExists(
                FileSystem fs,
                Path path,
                boolean expectedExists,
                long deadline) throws IOException {
    
        try (FsDataOutputStream ignored = fs.open(path)) {
                // object exists
                return true;
        }
        catch (FileNotFoundException e) {
                // object does not exist
                return false;
        }
    }
    ```
    instead of the current code.
    ```java
    public static void checkPathExistsEventually(
                FileSystem fs,
                Path path,
                boolean expectedExists,
                long deadline) throws IOException, InterruptedException {
        boolean dirExists;
        while ((dirExists = fs.exists(path)) != expectedExists &&
                        System.nanoTime() < deadline) {
                Thread.sleep(10);
        }
        assertEquals(expectedExists, dirExists);
    }
    ```
    
    What do you think? Should we use the "strongly consistent" variant for the 
Presto-S3-based connector?


> HadoopS3FileSystemITCase.testDirectoryListing fails on Travis
> -------------------------------------------------------------
>
>                 Key: FLINK-8402
>                 URL: https://issues.apache.org/jira/browse/FLINK-8402
>             Project: Flink
>          Issue Type: Bug
>          Components: FileSystem, Tests
>    Affects Versions: 1.5.0
>            Reporter: Till Rohrmann
>            Assignee: Nico Kruber
>            Priority: Blocker
>              Labels: test-stability
>             Fix For: 1.5.0, 1.4.3
>
>
> The test {{HadoopS3FileSystemITCase.testDirectoryListing}} fails on Travis.
> https://travis-ci.org/tillrohrmann/flink/jobs/327021175



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

Reply via email to