[
https://issues.apache.org/jira/browse/FLINK-4228?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16240265#comment-16240265
]
ASF GitHub Bot commented on FLINK-4228:
---------------------------------------
Github user tillrohrmann commented on a diff in the pull request:
https://github.com/apache/flink/pull/4939#discussion_r149070982
--- Diff: flink-yarn/src/main/java/org/apache/flink/yarn/Utils.java ---
@@ -117,27 +118,50 @@ public static void setupYarnClassPath(Configuration
conf, Map<String, String> ap
}
/**
+ * Copy a local file to a remote file system.
+ *
+ * @param fs
+ * remote filesystem
+ * @param appId
+ * application ID
+ * @param localRsrcPath
+ * path to the local file
+ * @param homedir
+ * remote home directory base (will be extended)
+ * @param relativeTargetPath
+ * relative target path of the file (will be prefixed be
the full home directory we set up)
+ *
* @return Path to remote file (usually hdfs)
- * @throws IOException
*/
- public static Path setupLocalResource(
- FileSystem fs,
- String appId, Path localRsrcPath,
- LocalResource appMasterJar,
- Path homedir) throws IOException {
+ static Tuple2<Path, LocalResource> setupLocalResource(
+ FileSystem fs,
+ String appId,
+ Path localRsrcPath,
+ Path homedir,
+ String relativeTargetPath) throws IOException {
+
+ if (new File(localRsrcPath.toUri().getPath()).isDirectory()) {
+ throw new IllegalArgumentException("File to copy must
not be a directory: " +
+ localRsrcPath);
+ }
// copy resource to HDFS
- String suffix = ".flink/" + appId + "/" +
localRsrcPath.getName();
+ String suffix = ".flink/" + appId + "/" + relativeTargetPath +
"/" + localRsrcPath.getName();
Path dst = new Path(homedir, suffix);
LOG.info("Copying from " + localRsrcPath + " to " + dst);
- fs.copyFromLocalFile(localRsrcPath, dst);
- registerLocalResource(fs, dst, appMasterJar);
- return dst;
+
+ fs.copyFromLocalFile(false, true, localRsrcPath, dst);
+
+ // now create the resource instance
+ LocalResource resource = Records.newRecord(LocalResource.class);
+ registerLocalResource(fs, dst, resource);
+ return Tuple2.of(dst, resource);
}
- public static void registerLocalResource(FileSystem fs, Path
remoteRsrcPath, LocalResource localResource) throws IOException {
+ private static void registerLocalResource(
+ FileSystem fs, Path remoteRsrcPath, LocalResource
localResource) throws IOException {
--- End diff --
Either put every parameter on a different line or please revert this change.
> 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)