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

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

GitHub user NicoK opened a pull request:

    https://github.com/apache/flink/pull/4939

    [FLINK-4228][yarn/s3a] fix yarn resource upload s3a defaultFs

    ## What is the purpose of the change
    
    If YARN is configured to use the `s3a` default file system, upload of the 
Flink jars will fail since its 
`org.apache.hadoop.fs.FileSystem#copyFromLocalFile()` does not work recursively 
on the given `lib` folder.
    
    ## Brief change log
    
    - implement our own recursive upload (based on #2288)
    - add unit tests to verify its behaviour for both `hdfs://` and `s3://` 
(via S3A) resource uploads
    
    ## Verifying this change
    
    This change added tests and can be verified as follows:
    
    - added a unit test for HDFS uploads via our `MiniDFSCluster`
    - added integration test to verify S3 uploads (via the S3A filesystem 
implementation of the `flink-s3-fs-hadoop` sub-project)
    - manually verified the test on YARN with both S3A and HDFS default file 
systems being set
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (yes - internally)
      - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: (no)
      - The serializers: (no)
      - The runtime per-record code paths (performance sensitive): (no)
      - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (no)
      - If yes, how is the feature documented? (JavaDocs)


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/NicoK/flink flink-4228

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/4939.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #4939
    
----
commit 5d31f41e0e480820e9fec1efa84e5725364a136d
Author: Nico Kruber <n...@data-artisans.com>
Date:   2017-11-02T18:38:48Z

    [hotfix][s3] fix HadoopS3FileSystemITCase leaving test directories behind 
in S3

commit bf47d376397a8e64625a031468d5f5d0a5486238
Author: Nico Kruber <n...@data-artisans.com>
Date:   2016-11-09T20:04:50Z

    [FLINK-4228][yarn/s3] fix for yarn staging with s3a defaultFs
    
    + includes a new unit tests for recursive uploads to hfds:// targets
    + add a unit test for recursive file uploads to s3:// via s3a

----


> YARN artifact upload does not work with S3AFileSystem
> -----------------------------------------------------
>
>                 Key: FLINK-4228
>                 URL: https://issues.apache.org/jira/browse/FLINK-4228
>             Project: Flink
>          Issue Type: Bug
>          Components: State Backends, Checkpointing
>            Reporter: Ufuk Celebi
>            Priority: Blocker
>             Fix For: 1.4.0
>
>
> The issue now is exclusive to running on YARN with s3a:// as your configured 
> FileSystem. If so, the Flink session will fail on staging itself because it 
> tries to copy the flink/lib directory to S3 and the S3aFileSystem does not 
> support recursive copy.
> h2. Old Issue
> Using the {{RocksDBStateBackend}} with semi-async snapshots (current default) 
> leads to an Exception when uploading the snapshot to S3 when using the 
> {{S3AFileSystem}}.
> {code}
> AsynchronousException{com.amazonaws.AmazonClientException: Unable to 
> calculate MD5 hash: 
> /var/folders/_c/5tc5q5q55qjcjtqwlwvwd1m00000gn/T/flink-io-5640e9f1-3ea4-4a0f-b4d9-3ce9fbd98d8a/7c6e745df2dddc6eb70def1240779e44/StreamFlatMap_3_0/dummy_state/47daaf2a-150c-4208-aa4b-409927e9e5b7/local-chk-2886
>  (Is a directory)}
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointThread.run(StreamTask.java:870)
> Caused by: com.amazonaws.AmazonClientException: Unable to calculate MD5 hash: 
> /var/folders/_c/5tc5q5q55qjcjtqwlwvwd1m00000gn/T/flink-io-5640e9f1-3ea4-4a0f-b4d9-3ce9fbd98d8a/7c6e745df2dddc6eb70def1240779e44/StreamFlatMap_3_0/dummy_state/47daaf2a-150c-4208-aa4b-409927e9e5b7/local-chk-2886
>  (Is a directory)
>       at 
> com.amazonaws.services.s3.AmazonS3Client.putObject(AmazonS3Client.java:1298)
>       at 
> com.amazonaws.services.s3.transfer.internal.UploadCallable.uploadInOneChunk(UploadCallable.java:108)
>       at 
> com.amazonaws.services.s3.transfer.internal.UploadCallable.call(UploadCallable.java:100)
>       at 
> com.amazonaws.services.s3.transfer.internal.UploadMonitor.upload(UploadMonitor.java:192)
>       at 
> com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:150)
>       at 
> com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:50)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>       at java.lang.Thread.run(Thread.java:745)
> Caused by: java.io.FileNotFoundException: 
> /var/folders/_c/5tc5q5q55qjcjtqwlwvwd1m00000gn/T/flink-io-5640e9f1-3ea4-4a0f-b4d9-3ce9fbd98d8a/7c6e745df2dddc6eb70def1240779e44/StreamFlatMap_3_0/dummy_state/47daaf2a-150c-4208-aa4b-409927e9e5b7/local-chk-2886
>  (Is a directory)
>       at java.io.FileInputStream.open0(Native Method)
>       at java.io.FileInputStream.open(FileInputStream.java:195)
>       at java.io.FileInputStream.<init>(FileInputStream.java:138)
>       at 
> com.amazonaws.services.s3.AmazonS3Client.putObject(AmazonS3Client.java:1294)
>       ... 9 more
> {code}
> Running with S3NFileSystem, the error does not occur. The problem might be 
> due to {{HDFSCopyToLocal}} assuming that sub-folders are going to be created 
> automatically. We might need to manually create folders and copy only actual 
> files for {{S3AFileSystem}}. More investigation is required.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to