[ https://issues.apache.org/jira/browse/FLINK-11086?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17089367#comment-17089367 ]
Robert Metzger commented on FLINK-11086: ---------------------------------------- Implementation plan: - wait for decision on the dev@ ml - replace all flink-shaded-hadoop-* dependencies with vanilla dependencies - remove shading from flink-swift-fs-hadoop (update docs) - test flink with hadoop manually - update documentation on how to use flink with hadoop > Add support for Hadoop 3 > ------------------------ > > Key: FLINK-11086 > URL: https://issues.apache.org/jira/browse/FLINK-11086 > Project: Flink > Issue Type: New Feature > Components: Deployment / YARN > Reporter: Sebastian Klemke > Assignee: Robert Metzger > Priority: Major > > All builds using maven 3.2.5 on commithash > ed8ff14ed39d08cd319efe75b40b9742a2ae7558. > Attempted builds: > - mvn clean install -Dhadoop.version=3.0.3 > - mvn clean install -Dhadoop.version=3.1.1 > Integration tests with Hadoop input format datasource fail. Example stack > trace, taken from hadoop.version 3.1.1 build: > {code:java} > testJobCollectionExecution(org.apache.flink.test.hadoopcompatibility.mapred.WordCountMapredITCase) > Time elapsed: 0.275 sec <<< ERR > OR! > java.lang.NoClassDefFoundError: > org/apache/flink/hadoop/shaded/com/google/re2j/PatternSyntaxException > at java.net.URLClassLoader.findClass(URLClassLoader.java:382) > at java.lang.ClassLoader.loadClass(ClassLoader.java:424) > at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:349) > at java.lang.ClassLoader.loadClass(ClassLoader.java:357) > at org.apache.hadoop.fs.Globber.doGlob(Globber.java:210) > at org.apache.hadoop.fs.Globber.glob(Globber.java:149) > at org.apache.hadoop.fs.FileSystem.globStatus(FileSystem.java:2085) > at > org.apache.hadoop.mapred.FileInputFormat.singleThreadedListStatus(FileInputFormat.java:269) > at > org.apache.hadoop.mapred.FileInputFormat.listStatus(FileInputFormat.java:239) > at > org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:325) > at > org.apache.flink.api.java.hadoop.mapred.HadoopInputFormatBase.createInputSplits(HadoopInputFormatBase.java:150) > at > org.apache.flink.api.java.hadoop.mapred.HadoopInputFormatBase.createInputSplits(HadoopInputFormatBase.java:58) > at > org.apache.flink.api.common.operators.GenericDataSourceBase.executeOnCollections(GenericDataSourceBase.java:225) > at > org.apache.flink.api.common.operators.CollectionExecutor.executeDataSource(CollectionExecutor.java:219) > at > org.apache.flink.api.common.operators.CollectionExecutor.execute(CollectionExecutor.java:155) > at > org.apache.flink.api.common.operators.CollectionExecutor.executeUnaryOperator(CollectionExecutor.java:229) > at > org.apache.flink.api.common.operators.CollectionExecutor.execute(CollectionExecutor.java:149) > at > org.apache.flink.api.common.operators.CollectionExecutor.executeUnaryOperator(CollectionExecutor.java:229) > at > org.apache.flink.api.common.operators.CollectionExecutor.execute(CollectionExecutor.java:149) > at > org.apache.flink.api.common.operators.CollectionExecutor.executeUnaryOperator(CollectionExecutor.java:229) > at > org.apache.flink.api.common.operators.CollectionExecutor.execute(CollectionExecutor.java:149) > at > org.apache.flink.api.common.operators.CollectionExecutor.executeUnaryOperator(CollectionExecutor.java:229) > at > org.apache.flink.api.common.operators.CollectionExecutor.execute(CollectionExecutor.java:149) > at > org.apache.flink.api.common.operators.CollectionExecutor.execute(CollectionExecutor.java:131) > at > org.apache.flink.api.common.operators.CollectionExecutor.executeDataSink(CollectionExecutor.java:182) > at > org.apache.flink.api.common.operators.CollectionExecutor.execute(CollectionExecutor.java:158) > at > org.apache.flink.api.common.operators.CollectionExecutor.execute(CollectionExecutor.java:131) > at > org.apache.flink.api.common.operators.CollectionExecutor.execute(CollectionExecutor.java:115) > at > org.apache.flink.api.java.CollectionEnvironment.execute(CollectionEnvironment.java:38) > at > org.apache.flink.test.util.CollectionTestEnvironment.execute(CollectionTestEnvironment.java:52) > at > org.apache.flink.test.hadoopcompatibility.mapred.WordCountMapredITCase.internalRun(WordCountMapredITCase.java:121) > at > org.apache.flink.test.hadoopcompatibility.mapred.WordCountMapredITCase.testProgram(WordCountMapredITCase.java:71) > {code} > Maybe hadoop 3.x versions could be added to test matrix as well? -- This message was sent by Atlassian Jira (v8.3.4#803005)