You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2019/11/20 11:10:00 UTC

[jira] [Updated] (FLINK-14876) Putting xercesImpl related classes into alwaysParentFirstLoaderPatterns

     [ https://issues.apache.org/jira/browse/FLINK-14876?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

ASF GitHub Bot updated FLINK-14876:
-----------------------------------
    Labels: pull-request-available  (was: )

> Putting xercesImpl related classes into alwaysParentFirstLoaderPatterns
> -----------------------------------------------------------------------
>
>                 Key: FLINK-14876
>                 URL: https://issues.apache.org/jira/browse/FLINK-14876
>             Project: Flink
>          Issue Type: Improvement
>          Components: Runtime / Configuration
>    Affects Versions: 1.9.0
>         Environment: hadoop2.6
>  
>            Reporter: liupengcheng
>            Priority: Major
>              Labels: pull-request-available
>
> As mentioned in the [9683|[https://github.com/apache/flink/pull/9683]]  , There is a issue when running jobs when including flink-shaded-hadoop-2 package while changing the `hadoop.version` to our cluster hadoop version.
> I reproducing this case, the following exception was reported:
> {code:java}
> javax.xml.parsers.FactoryConfigurationError: Provider for class javax.xml.parsers.DocumentBuilderFactory cannot be createdjavax.xml.parsers.FactoryConfigurationError: Provider for class javax.xml.parsers.DocumentBuilderFactory cannot be created at javax.xml.parsers.FactoryFinder.findServiceProvider(FactoryFinder.java:311) at javax.xml.parsers.FactoryFinder.find(FactoryFinder.java:267) at javax.xml.parsers.DocumentBuilderFactory.newInstance(DocumentBuilderFactory.java:120) at org.apache.hadoop.conf.Configuration.loadResource(Configuration.java:2412) at org.apache.hadoop.conf.Configuration.loadResources(Configuration.java:2375) at org.apache.hadoop.conf.Configuration.getProps(Configuration.java:2285) at org.apache.hadoop.conf.Configuration.get(Configuration.java:892) at org.apache.hadoop.mapred.JobConf.checkAndWarnDeprecation(JobConf.java:2010) at org.apache.hadoop.mapred.JobConf.<init>(JobConf.java:449) at org.apache.hadoop.mapreduce.Job.getInstance(Job.java:186) at org.apache.hadoop.mapreduce.Job.getInstance(Job.java:167) at org.apache.flink.hadoopcompatibility.scala.HadoopInputs$.readHadoopFile(HadoopInputs.scala:127) at com.github.ehiggs.spark.terasort.FlinkTeraSort$.main(FlinkTeraSort.scala:76) at com.github.ehiggs.spark.terasort.FlinkTeraSort.main(FlinkTeraSort.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:586) at org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:448) at org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:274) at org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:746) at org.apache.flink.client.cli.CliFrontend.runProgram(CliFrontend.java:273) at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:205) at org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:1009) at org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:1082) 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:1886) at org.apache.flink.runtime.security.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41) at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:1082)Caused by: java.lang.RuntimeException: Provider for class javax.xml.parsers.DocumentBuilderFactory cannot be created at javax.xml.parsers.FactoryFinder.findServiceProvider(FactoryFinder.java:308) ... 30 moreCaused by: java.util.ServiceConfigurationError: javax.xml.parsers.DocumentBuilderFactory: Provider org.apache.xerces.jaxp.DocumentBuilderFactoryImpl not a subtype at java.util.ServiceLoader.fail(ServiceLoader.java:239) at java.util.ServiceLoader.access$300(ServiceLoader.java:185) at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:376) at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404) at java.util.ServiceLoader$1.next(ServiceLoader.java:480) at javax.xml.parsers.FactoryFinder$1.run(FactoryFinder.java:294) at java.security.AccessController.doPrivileged(Native Method) at javax.xml.parsers.FactoryFinder.findServiceProvider(FactoryFinder.java:289) ... 30 more
> {code}
> I enabled `-vebose:class` for client, and find out that the `DocumentBuilderFactoryImpl` class was loaded twice. The first time it was loaded is because the flink app starts and initialize the configuration and print something, the second time it was loaded is because the calling of `HadoopInputs.readHadoopFile` in the user code.
> {code:java}
> [Loaded javax.xml.parsers.DocumentBuilderFactory from /opt/soft/openjdk8u202-b08/jre/lib/rt.jar]
> [Loaded org.apache.xerces.jaxp.DocumentBuilderFactoryImpl from file:/home/liupengcheng/git/infra-client/bin/packages/common-infra_client-pack-zjyprc-hadoop/bin/packages/zjyprc-hadoop-flink1.9-hadoop-pack-2.6.0-mdh2.6.0.4/share/hadoop/hdfs/lib/xercesImpl-2.9.1.jar]
> [Loaded javax.xml.parsers.DocumentBuilderFactory from file:/home/liupengcheng/git/infra-client/bin/packages/common-infra_client-pack-zjyprc-hadoop/bin/packages/flink-1.9.0-mdh1.9.0.0-SNAPSHOT/../benchmark-test-1.1-SNAPSHOT-shaded.jar]
> [Loaded org.apache.xerces.jaxp.DocumentBuilderFactoryImpl from file:/home/liupengcheng/git/infra-client/bin/packages/common-infra_client-pack-zjyprc-hadoop/bin/packages/flink-1.9.0-mdh1.9.0.0-SNAPSHOT/../benchmark-test-1.1-SNAPSHOT-shaded.jar]
> {code}
> {code:java}
> val dataSet = env.createInput(HadoopInputs.readHadoopFile(
>   new TeraInputFormat, classOf[Array[Byte]], classOf[Array[Byte]], inputFile))
>   .partitionCustom(new FlinkTeraSortPartitioner(new TeraSortPartitioner(partitions)), 0)
>   .sortPartition(0, Order.ASCENDING)
> {code}
> when calling `HadoopInputs.readhadoopFile`, it will instantiate `jobConf` and call it's `get` method, thus will finally causing the loading of `DocumentBuilderFactoryImpl`.
> After loading `DocumentBuilderFactoryImpl`, it will try to check whether it's subtype of `DocumentBuilderFactory`. Here, it' will report error due to that the `DocumentBuilderFacotory` is loaded from `HADOOP_CLASSPATH`, not the user code.
> {code:java}
> public static DocumentBuilderFactory newInstance() {
>     return FactoryFinder.find(
>             /* The default property name according to the JAXP spec */
>             DocumentBuilderFactory.class, // "javax.xml.parsers.DocumentBuilderFactory"
>             /* The fallback implementation class name */
>             "com.sun.org.apache.xerces.internal.jaxp.DocumentBuilderFactoryImpl");
> }
> {code}
> I found that  the `xercesImpl` package was introduced by the `flink-shaded-hadoop-2`, because it shaded in `xercesImpl` from hadoop-hdfs deps.
>  
> Moreover, this `xcercesImpl` is included in hadoop-hdfs for all 2.6 ~ 3.0 versions, I think we can add it to the `alwaysParentFirstPatterns` to avoid this problem. 
> cc [~aljoscha]



--
This message was sent by Atlassian Jira
(v8.3.4#803005)