flink-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Vijay Srinivasaraghavan <vijikar...@yahoo.com.INVALID>
Subject Re: Flink 1.4.0-RC2 Hadoop Build
Date Mon, 04 Dec 2017 17:45:40 GMT
Hi Aljoscha,
>> Regarding the issue you discovered, I think we have to revert a commit that added
exclusions that were too aggressive.

Is there a JIRA for this?  
RegardsVijay

    On Monday, December 4, 2017 1:16 AM, Aljoscha Krettek <aljoscha@apache.org> wrote:
 

 Hi,

The behaviour with Flink 1.4 is that we will always include Hadoop dependencies unless we
explicitly exclude them via "-DwithoutHadoop". If you build without any flags you will get
the default Hadoop version which currently is 2.4.1.

Regarding the issue you discovered, I think we have to revert a commit that added exclusions
that were too aggressive.

Best,
Aljoscha

> On 3. Dec 2017, at 21:58, Vijay Srinivasaraghavan <vijikarthi@yahoo.com.INVALID>
wrote:
> 
> The issue is reproducible with 2.7.1 as well. 
> 
> My understanding is from 1.4.0 we don't include Hadoop dependencies by default but only
when we explicitly enable the particular Hadoop version. However, when I build it normally
using "mvn clean install -DskipTests", the Hadoop dependencies are getting included and I
did not see the error that I have reported earlier.
> 
> Regards,
> Vijay
> 
> Sent from my iPhone
> 
>> On Dec 3, 2017, at 11:23 AM, Stephan Ewen <sewen@apache.org> wrote:
>> 
>> Hmmm, this is caused by a missing dependency (javax.servlet). 
>> Could be that this dependency is newly excluded, or that this issue only exists with
Hadoop 2.7.0 (HDFS release tests have worked with both newer and older Hadoop versions).
>> 
>> Here is what I think we should do
>> 
>>  - Can you check whether this works with other Hadoop versions (for example Hadoop
2.7.1 or so?)
>>  - To run this in practice, you can also just drop the bundled hadoop dependency
and use whatever Hadoop dependency you have already on the cluster (for example via yarn)
>>  - Flink needs to adjust some dependencies / exclusions for the pre-bundled Hadoop
to make this build error go away
>> 
>> 
>> In any case, the fact that Hadoop seriously needs the Servlet dependency to initialize
HDFS is just ridiculous. Hadoop seriously has the worst dependency management I have ever
seen.
>> 
>> 
>>> On Fri, Dec 1, 2017 at 1:41 PM, Vijay Srinivasaraghavan <vijikarthi@yahoo.com.invalid>
wrote:
>>> Hello,
>>> I am trying to build and run Flink from 1.4.0-rc2 branch with hadoop binary 2.7.0
compatibility.
>>> Here are the steps I followed to build (I have maven 3.3.9).
>>> ===========================================================================cd
$FLINK_HOMEmvn clean install -DskipTests -Dhadoop.version=2.7.0cd $FLINK_HOME/flink-distmvn
clean install -Dhadoop.version=2.7.0
>>> Running Flink from $FLINK_HOME/flink-dist/target/flink-1.4.0-bin==========================================================================
>>> I am seeing below error error messages from the logs and it suggests something
to do with Hadoop dependency not available?
>>> Could someone please confirm the build steps?
>>> org.apache.flink.core.fs.UnsupportedFileSystemSchemeException: Could not find
a file system implementation for scheme 'hdfs'. The scheme is not directly supported by Flink
and no Hadoop file system to support this scheme could be loaded. at org.apache.flink.core.fs.FileSystem.getUnguardedFileSystem(FileSystem.java:403)
at org.apache.flink.core.fs.FileSystem.get(FileSystem.java:318) at org.apache.flink.core.fs.Path.getFileSystem(Path.java:293)
at org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory.<init>(FsCheckpointStreamFactory.java:99)
at org.apache.flink.runtime.state.filesystem.FsStateBackend.createStreamFactory(FsStateBackend.java:277)
at org.apache.flink.streaming.runtime.tasks.StreamTask.createCheckpointStreamFactory(StreamTask.java:787)
at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:247)
at org.apache.flink.streaming.runtime.tasks.StreamTask.initializeOperators(StreamTask.java:694)
at org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:682)
at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:253) at org.apache.flink.runtime.taskmanager.Task.run(Task.java:718)
at java.lang.Thread.run(Thread.java:748)Caused by: org.apache.flink.core.fs.UnsupportedFileSystemSchemeException:
Cannot support file system for 'hdfs' via Hadoop, because Hadoop is not in the classpath,
or some classes are missing from the classpath. at org.apache.flink.runtime.fs.hdfs.HadoopFsFactory.create(HadoopFsFactory.java:179)
at org.apache.flink.core.fs.FileSystem.getUnguardedFileSystem(FileSystem.java:399) ... 11
moreCaused by: java.lang.NoClassDefFoundError: Could not initialize class org.apache.hadoop.hdfs.DFSConfigKeys
at org.apache.hadoop.hdfs.DFSClient$Conf.<init>(DFSClient.java:509) at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:637)
at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:619) at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:149)
at org.apache.flink.runtime.fs.hdfs.HadoopFsFactory.create(HadoopFsFactory.java:159) ... 12
more2017-12-01 04:28:09,274 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph 
      - Source: Collection Source -> Map (1/1) (74e8a3f21c86e8dec3c55988fff42e5d) switched
from RUNNING to FAILED.org.apache.flink.core.fs.UnsupportedFileSystemSchemeException: Could
not find a file system implementation for scheme 'hdfs'. The scheme is not directly supported
by Flink and no Hadoop file system to support this scheme could be loaded. at org.apache.flink.core.fs.FileSystem.getUnguardedFileSystem(FileSystem.java:403)
at org.apache.flink.core.fs.FileSystem.get(FileSystem.java:318) at org.apache.flink.core.fs.Path.getFileSystem(Path.java:293)
at org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory.<init>(FsCheckpointStreamFactory.java:99)
at org.apache.flink.runtime.state.filesystem.FsStateBackend.createStreamFactory(FsStateBackend.java:277)
at org.apache.flink.streaming.runtime.tasks.StreamTask.createCheckpointStreamFactory(StreamTask.java:787)
at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:247)
at org.apache.flink.streaming.runtime.tasks.StreamTask.initializeOperators(StreamTask.java:694)
at org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:682)
at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:253) at org.apache.flink.runtime.taskmanager.Task.run(Task.java:718)
at java.lang.Thread.run(Thread.java:748)Caused by: org.apache.flink.core.fs.UnsupportedFileSystemSchemeException:
Cannot support file system for 'hdfs' via Hadoop, because Hadoop is not in the classpath,
or some classes are missing from the classpath. at org.apache.flink.runtime.fs.hdfs.HadoopFsFactory.create(HadoopFsFactory.java:179)
at org.apache.flink.core.fs.FileSystem.getUnguardedFileSystem(FileSystem.java:399) ... 11
moreCaused by: java.lang.NoClassDefFoundError: Could not initialize class org.apache.hadoop.hdfs.DFSConfigKeys
at org.apache.hadoop.hdfs.DFSClient$Conf.<init>(DFSClient.java:509) at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:637)
at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:619) at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:149)
at org.apache.flink.runtime.fs.hdfs.HadoopFsFactory.create(HadoopFsFactory.java:159) ... 12
more2017-12-01 04:28:09,274 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph 
      - Source: Collection Source -> Map (1/1) (18fcafbd7bc31a9257de7728b46cee7b) switched
from RUNNING to FAILED.org.apache.flink.core.fs.UnsupportedFileSystemSchemeException: Could
not find a file system implementation for scheme 'hdfs'. The scheme is not directly supported
by Flink and no Hadoop file system to support this scheme could be loaded. at org.apache.flink.core.fs.FileSystem.getUnguardedFileSystem(FileSystem.java:403)
at org.apache.flink.core.fs.FileSystem.get(FileSystem.java:318) at org.apache.flink.core.fs.Path.getFileSystem(Path.java:293)
at org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory.<init>(FsCheckpointStreamFactory.java:99)
at org.apache.flink.runtime.state.filesystem.FsStateBackend.createStreamFactory(FsStateBackend.java:277)
at org.apache.flink.streaming.runtime.tasks.StreamTask.createCheckpointStreamFactory(StreamTask.java:787)
at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:247)
at org.apache.flink.streaming.runtime.tasks.StreamTask.initializeOperators(StreamTask.java:694)
at org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:682)
at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:253) at org.apache.flink.runtime.taskmanager.Task.run(Task.java:718)
at java.lang.Thread.run(Thread.java:748)Caused by: org.apache.flink.core.fs.UnsupportedFileSystemSchemeException:
Cannot support file system for 'hdfs' via Hadoop, because Hadoop is not in the classpath,
or some classes are missing from the classpath. at org.apache.flink.runtime.fs.hdfs.HadoopFsFactory.create(HadoopFsFactory.java:179)
at org.apache.flink.core.fs.FileSystem.getUnguardedFileSystem(FileSystem.java:399) ... 11
moreCaused by: java.lang.NoClassDefFoundError: javax/servlet/Filter at java.lang.ClassLoader.defineClass1(Native
Method) at java.lang.ClassLoader.defineClass(ClassLoader.java:763) at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142)
at java.net.URLClassLoader.defineClass(URLClassLoader.java:467) at java.net.URLClassLoader.access$100(URLClassLoader.java:73)
at java.net.URLClassLoader$1.run(URLClassLoader.java:368) at java.net.URLClassLoader$1.run(URLClassLoader.java:362)
at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:361)
at java.lang.ClassLoader.loadClass(ClassLoader.java:424) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:335)
at java.lang.ClassLoader.loadClass(ClassLoader.java:357) at java.lang.ClassLoader.defineClass1(Native
Method) at java.lang.ClassLoader.defineClass(ClassLoader.java:763) at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142)
at java.net.URLClassLoader.defineClass(URLClassLoader.java:467) at java.net.URLClassLoader.access$100(URLClassLoader.java:73)
at java.net.URLClassLoader$1.run(URLClassLoader.java:368) at java.net.URLClassLoader$1.run(URLClassLoader.java:362)
at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:361)
at java.lang.ClassLoader.loadClass(ClassLoader.java:424) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:335)
at java.lang.ClassLoader.loadClass(ClassLoader.java:357) at org.apache.hadoop.hdfs.DFSConfigKeys.<clinit>(DFSConfigKeys.java:230)
at org.apache.hadoop.hdfs.DFSClient$Conf.<init>(DFSClient.java:509) at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:637)
at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:619) at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:149)
at org.apache.flink.runtime.fs.hdfs.HadoopFsFactory.create(HadoopFsFactory.java:159) ... 12
moreCaused by: java.lang.ClassNotFoundException: javax.servlet.Filter at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
at java.lang.ClassLoader.loadClass(ClassLoader.java:424) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:335)
at java.lang.ClassLoader.loadClass(ClassLoader.java:357) ... 42 more
>>> RegardsVijay
>> 


   
Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message