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

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

GitHub user jfg9 opened a pull request:

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

    [FLINK-4115] FsStateBackend filesystem verification can cause classpath 
exceptions

    There are two changes to FsStateBackend to avoid classpath exceptions when 
submitting a job from a Flink client which does not have the necessary file 
system classes on its classpath:
    
    - The filesystem is no longer initialised in the FsStateBackend constructor 
    - The verification checks for the checkpoint directory in the 
FsStateBackend constructor are skipped if Flink does not have built-in support 
for the URI scheme

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

    $ git pull https://github.com/jfg9/flink master

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

    https://github.com/apache/flink/pull/2157.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 #2157
    
----
commit e8557895b4ff8e40a0705152ec65d355fe7d5f65
Author: Josh <[email protected]>
Date:   2016-06-23T21:18:02Z

    [FLINK-4115] Skip filesystem checks for filesystems with no built-in support

commit 94ec8dca6c016c1c500ade90b4c823e5faaecaf3
Author: Josh <[email protected]>
Date:   2016-06-23T21:22:40Z

    [FLINK-4115] Remove filesystem initialisation from FsStateBackend 
constructor

----


> FsStateBackend filesystem verification can cause classpath exceptions
> ---------------------------------------------------------------------
>
>                 Key: FLINK-4115
>                 URL: https://issues.apache.org/jira/browse/FLINK-4115
>             Project: Flink
>          Issue Type: Bug
>          Components: Local Runtime
>    Affects Versions: 1.1.0
>            Reporter: Josh Forman-Gornall
>            Priority: Minor
>
> In the constructor of FsStateBackend, the FileSystem for the checkpoint 
> directory is initialised and it is verified that the checkpoint path exists. 
> This verification happens in the Flink client program when submitting a job 
> and can cause classpath issues if classes required to access the file system 
> are not available in the client's classpath.
> For example, if we run Flink on YARN over AWS EMR using RocksDBStateBackend 
> and an s3:// checkpoint directory, we get the below ClassNotFoundException. 
> This is because the jars needed to use the EMR file system are available only 
> in the YARN context and not when submitting the job via the Flink client.
> {noformat}
> java.lang.RuntimeException: java.lang.ClassNotFoundException: Class 
> com.amazon.ws.emr.hadoop.fs.EmrFileSystem not found
>       at 
> org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2227)
>       at 
> org.apache.flink.runtime.fs.hdfs.HadoopFileSystem.getHadoopWrapperClassNameForFileSystem(HadoopFileSystem.java:460)
>       at 
> org.apache.flink.core.fs.FileSystem.getHadoopWrapperClassNameForFileSystem(FileSystem.java:352)
>       at org.apache.flink.core.fs.FileSystem.get(FileSystem.java:280)
>       at 
> org.apache.flink.runtime.state.filesystem.FsStateBackend.validateAndNormalizeUri(FsStateBackend.java:383)
>       at 
> org.apache.flink.runtime.state.filesystem.FsStateBackend.<init>(FsStateBackend.java:175)
>       at 
> org.apache.flink.runtime.state.filesystem.FsStateBackend.<init>(FsStateBackend.java:144)
>       at 
> org.apache.flink.contrib.streaming.state.RocksDBStateBackend.<init>(RocksDBStateBackend.java:205)
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to