flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Nico Kruber (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-4228) YARN artifact upload does not work with S3AFileSystem
Date Mon, 30 Oct 2017 17:47:02 GMT

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

Nico Kruber commented on FLINK-4228:
------------------------------------

[~aljoscha]: I successfully reproduced the error with a current snapshot of Flink 1.4 on EMR

Steps to reproduce:
# setup the S3A filesystem as described in https://ci.apache.org/projects/flink/flink-docs-release-1.4/ops/deployment/aws.html,
i.e.
#* adapt {{/etc/hadoop/conf/core-site.xml}}
#* copy S3A filesystem dependency jars to {{flink/lib}}
# adapt default filesystem for hadoop in {{/etc/hadoop/conf/core-site.xml}}, e.g.:
{code}
  <property>
    <name>fs.defaultFS</name>
    <value>s3a://nico-test/</value>
  </property>
{code}
# distribute new configuration to all nodes & restart to apply
# try to run an example (here with two nodes) so that Flink tries to deploy the artefacts
to YARN's (default) filesystem
{code}
> cd flink
> HADOOP_CONF_DIR=/etc/hadoop/conf ./bin/flink run -m yarn-cluster -yn 2 -ys 1 -yjm 768
-ytm 1024 ./examples/batch/WordCount.jar
Using the result of 'hadoop classpath' to augment the Hadoop classpath: /etc/hadoop/conf:/usr/lib/hadoop/lib/*:/usr/lib/hadoop/.//*:/usr/lib/hadoop-hdfs/./:/usr/lib/hadoop-hdfs/lib/*:/usr/lib/hadoop-hdfs/.//*:/usr/lib/hadoop-yarn/lib/*:/usr/lib/hadoop-yarn/.//*:/usr/lib/hadoop-mapreduce/lib/*:/usr/lib/hadoop-mapreduce/.//*::/usr/lib/hadoop-lzo/lib/*:/usr/share/aws/aws-java-sdk/*:/usr/share/aws/emr/emrfs/conf:/usr/share/aws/emr/emrfs/lib/*:/usr/share/aws/emr/emrfs/auxlib/*:/usr/share/aws/emr/ddb/lib/emr-ddb-hadoop.jar:/usr/share/aws/emr/goodies/lib/emr-hadoop-goodies.jar:/usr/share/aws/emr/kinesis/lib/emr-kinesis-hadoop.jar:/usr/share/aws/emr/cloudwatch-sink/lib/*:/usr/share/aws/emr/security/conf:/usr/share/aws/emr/security/lib/*
SLF4J: Class path contains multiple SLF4J bindings.
SLF4J: Found binding in [jar:file:/home/hadoop/flink/lib/slf4j-log4j12-1.7.7.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in [jar:file:/usr/lib/hadoop/lib/slf4j-log4j12-1.7.10.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an explanation.
SLF4J: Actual binding is of type [org.slf4j.impl.Log4jLoggerFactory]
2017-10-30 17:34:51,137 WARN  org.apache.hadoop.conf.Configuration                       
  - /etc/hadoop/conf/core-site.xml:an attempt to override final parameter: fs.s3.buffer.dir;
 Ignoring.
2017-10-30 17:34:51,224 INFO  org.apache.flink.yarn.cli.FlinkYarnSessionCli              
  - No path for the flink jar passed. Using the location of class org.apache.flink.yarn.YarnClusterDescriptor
to locate the jar
2017-10-30 17:34:51,224 INFO  org.apache.flink.yarn.cli.FlinkYarnSessionCli              
  - No path for the flink jar passed. Using the location of class org.apache.flink.yarn.YarnClusterDescriptor
to locate the jar
2017-10-30 17:34:51,279 INFO  org.apache.hadoop.yarn.client.RMProxy                      
  - Connecting to ResourceManager at ip-172-31-22-149.eu-west-1.compute.internal/172.31.22.149:8032
2017-10-30 17:34:51,572 INFO  org.apache.flink.yarn.YarnClusterDescriptor                
  - Cluster specification: ClusterSpecification{masterMemoryMB=768, taskManagerMemoryMB=1024,
numberTaskManagers=2, slotsPerTaskManager=1}
2017-10-30 17:34:53,253 WARN  org.apache.flink.yarn.YarnClusterDescriptor                
  - The configuration directory ('/home/hadoop/flink/conf') contains both LOG4J and Logback
configuration files. Please delete or rename one of them.
2017-10-30 17:34:53,268 INFO  org.apache.flink.yarn.Utils                                
  - Copying from file:/home/hadoop/flink/conf/log4j.properties to s3a://nico-test/user/hadoop/.flink/application_1509384765476_0001/log4j.properties
2017-10-30 17:34:53,824 INFO  org.apache.flink.yarn.Utils                                
  - Copying from file:/home/hadoop/flink/lib to s3a://nico-test/user/hadoop/.flink/application_1509384765476_0001/lib

------------------------------------------------------------
 The program finished with the following exception:

java.lang.RuntimeException: Error deploying the YARN cluster
        at org.apache.flink.yarn.cli.FlinkYarnSessionCli.createCluster(FlinkYarnSessionCli.java:594)
        at org.apache.flink.yarn.cli.FlinkYarnSessionCli.createCluster(FlinkYarnSessionCli.java:81)
        at org.apache.flink.client.CliFrontend.createClient(CliFrontend.java:925)
        at org.apache.flink.client.CliFrontend.run(CliFrontend.java:264)
        at org.apache.flink.client.CliFrontend.parseParameters(CliFrontend.java:1054)
        at org.apache.flink.client.CliFrontend$1.call(CliFrontend.java:1101)
        at org.apache.flink.client.CliFrontend$1.call(CliFrontend.java:1098)
        at java.security.AccessController.doPrivileged(Native Method)
        at javax.security.auth.Subject.doAs(Subject.java:422)
        at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1556)
        at org.apache.flink.runtime.security.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41)
        at org.apache.flink.client.CliFrontend.main(CliFrontend.java:1098)
Caused by: java.lang.RuntimeException: Couldn't deploy Yarn session cluster
        at org.apache.flink.yarn.AbstractYarnClusterDescriptor.deploySessionCluster(AbstractYarnClusterDescriptor.java:368)
        at org.apache.flink.yarn.cli.FlinkYarnSessionCli.createCluster(FlinkYarnSessionCli.java:592)
        ... 11 more
Caused by: com.amazonaws.SdkClientException: Unable to calculate MD5 hash: /home/hadoop/flink/lib
(Is a directory)
        at com.amazonaws.services.s3.AmazonS3Client.putObject(AmazonS3Client.java:1624)
        at com.amazonaws.services.s3.transfer.internal.UploadCallable.uploadInOneChunk(UploadCallable.java:133)
        at com.amazonaws.services.s3.transfer.internal.UploadCallable.call(UploadCallable.java:125)
        at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:143)
        at com.amazonaws.services.s3.transfer.internal.UploadMonitor.call(UploadMonitor.java:48)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
Caused by: java.io.FileNotFoundException: /home/hadoop/flink/lib (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.util.Md5Utils.computeMD5Hash(Md5Utils.java:97)
        at com.amazonaws.util.Md5Utils.md5AsBase64(Md5Utils.java:104)
        at com.amazonaws.services.s3.AmazonS3Client.putObject(AmazonS3Client.java:1620)
        ... 8 more
{code}

> 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)

Mime
View raw message