Mate Czagany created FLINK-30812:
------------------------------------
Summary: YARN with S3 resource storage fails for Hadoop 3.3.2
Key: FLINK-30812
URL: https://issues.apache.org/jira/browse/FLINK-30812
Project: Flink
Issue Type: Bug
Components: Deployment / YARN
Affects Versions: 1.16.0
Reporter: Mate Czagany
In HADOOP-17139 S3AFileSystem#copyFromLocalFile was refactored and expects the
local source Hadoop Path object to have a scheme specified which the
YarnClusterDescriptor uploading the local files won't have.
When uploading files to S3 CopyFromLocalOperation#getFinalPath compares the
passed source Hadoop Path with the file it found(which will have file://
scheme) using URI.relativize but it will fail because of the scheme difference
and throw PathIOException as can be seen in this exception:
{code:java}
org.apache.flink.client.deployment.ClusterDeploymentException: Couldn't deploy
Yarn Application Cluster
at
org.apache.flink.yarn.YarnClusterDescriptor.deployApplicationCluster(YarnClusterDescriptor.java:478)
~[flink-yarn-1.16.0.jar!/:1.16.0] ......
Caused by: org.apache.hadoop.fs.PathIOException: `Cannot get relative path for
URI:file:///tmp/application_1674531932229_0030-flink-conf.yaml587547081521530798.tmp':
Input/output error
at
org.apache.hadoop.fs.s3a.impl.CopyFromLocalOperation.getFinalPath(CopyFromLocalOperation.java:360)
~[flink-s3-fs-hadoop-1.16.0.jar!/:1.16.0]
at
org.apache.hadoop.fs.s3a.impl.CopyFromLocalOperation.uploadSourceFromFS(CopyFromLocalOperation.java:222)
~[flink-s3-fs-hadoop-1.16.0.jar!/:1.16.0]
at
org.apache.hadoop.fs.s3a.impl.CopyFromLocalOperation.execute(CopyFromLocalOperation.java:169)
~[flink-s3-fs-hadoop-1.16.0.jar!/:1.16.0]
at
org.apache.hadoop.fs.s3a.S3AFileSystem.lambda$copyFromLocalFile$25(S3AFileSystem.java:3920)
~[flink-s3-fs-hadoop-1.16.0.jar!/:1.16.0]
at
org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.lambda$trackDurationOfOperation$5(IOStatisticsBinding.java:499)
~[hadoop-common-3.3.3.jar!/:?]
at
org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration(IOStatisticsBinding.java:444)
~[hadoop-common-3.3.3.jar!/:?]
at
org.apache.hadoop.fs.s3a.S3AFileSystem.trackDurationAndSpan(S3AFileSystem.java:2337)
~[flink-s3-fs-hadoop-1.16.0.jar!/:1.16.0]
at
org.apache.hadoop.fs.s3a.S3AFileSystem.trackDurationAndSpan(S3AFileSystem.java:2356)
~[flink-s3-fs-hadoop-1.16.0.jar!/:1.16.0]
at
org.apache.hadoop.fs.s3a.S3AFileSystem.copyFromLocalFile(S3AFileSystem.java:3913)
~[flink-s3-fs-hadoop-1.16.0.jar!/:1.16.0]
at
org.apache.flink.yarn.YarnApplicationFileUploader.copyToRemoteApplicationDir(YarnApplicationFileUploader.java:397)
~[flink-yarn-1.16.0.jar!/:1.16.0]
at
org.apache.flink.yarn.YarnApplicationFileUploader.uploadLocalFileToRemote(YarnApplicationFileUploader.java:202)
~[flink-yarn-1.16.0.jar!/:1.16.0]
at
org.apache.flink.yarn.YarnApplicationFileUploader.registerSingleLocalResource(YarnApplicationFileUploader.java:181)
~[flink-yarn-1.16.0.jar!/:1.16.0]
at
org.apache.flink.yarn.YarnClusterDescriptor.startAppMaster(YarnClusterDescriptor.java:1047)
~[flink-yarn-1.16.0.jar!/:1.16.0]
at
org.apache.flink.yarn.YarnClusterDescriptor.deployInternal(YarnClusterDescriptor.java:623)
~[flink-yarn-1.16.0.jar!/:1.16.0]
at
org.apache.flink.yarn.YarnClusterDescriptor.deployApplicationCluster(YarnClusterDescriptor.java:471)
~[flink-yarn-1.16.0.jar!/:1.16.0]
... 35 more {code}
The possibly easiest solution would be to somehow add the file:// scheme in
YarnApplicationFileUploader#copyToRemoteApplicationDir
The other solution would be to change all calls uploading local files to use
"new Path(file.toURI())" instead of "new Path(file.getAbsolutePath())" but it
might not be as future-proof as the other solution
Email thread: [https://lists.apache.org/thread/oo5rlyo3jr7kds2y6wwnfo1yhnk0fx4c]
If a committer can assign this ticket to me I can start working on this
--
This message was sent by Atlassian Jira
(v8.20.10#820010)