Skip to content

Conversation

@liupc
Copy link
Contributor

@liupc liupc commented Nov 20, 2019

What is the purpose of the change

There is a issue when running jobs when including flink-shaded-hadoop-2 package while changing the hadoop.version to our cluster hadoop version(2.6), more details see https://issues.apache.org/jira/browse/FLINK-14876

javax.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 more
Caused 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

This PR is to fix this problem by adding xercesImpl related classes into alwaysParentFirstLoaderPatterns.

Brief change log

  • Add xercesImpl related classes into alwaysParentFirstLoaderPatterns.

Verifying this change

This change added tests and can be verified as follows:

  • Manually verified the change by running app in yarn-cluster mode

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): (no)
  • The public API, i.e., is any changed class annotated with @Public(Evolving): (no)
  • The serializers: (no)
  • The runtime per-record code paths (performance sensitive): (no)
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (no)
  • The S3 file system connector: (no)

Documentation

  • Does this pull request introduce a new feature? (no)
  • If yes, how is the feature documented? (not applicable)

@liupc
Copy link
Contributor Author

liupc commented Nov 20, 2019

cc @aljoscha

@flinkbot
Copy link
Collaborator

Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
to review your pull request. We will use this comment to track the progress of the review.

Automated Checks

Last check on commit caaddea (Wed Nov 20 11:12:10 UTC 2019)

Warnings:

  • No documentation files were touched! Remember to keep the Flink docs up to date!
  • This pull request references an unassigned Jira ticket. According to the code contribution guide, tickets need to be assigned before starting with the implementation work.

Mention the bot in a comment to re-run the automated checks.

Review Progress

  • ❓ 1. The [description] looks good.
  • ❓ 2. There is [consensus] that the contribution should go into to Flink.
  • ❓ 3. Needs [attention] from.
  • ❓ 4. The change fits into the overall [architecture].
  • ❓ 5. Overall code [quality] is good.

Please see the Pull Request Review Guide for a full explanation of the review process.

Details
The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required Bot commands
The @flinkbot bot supports the following commands:

  • @flinkbot approve description to approve one or more aspects (aspects: description, consensus, architecture and quality)
  • @flinkbot approve all to approve all aspects
  • @flinkbot approve-until architecture to approve everything until architecture
  • @flinkbot attention @username1 [@username2 ..] to require somebody's attention
  • @flinkbot disapprove architecture to remove an approval you gave earlier

@flinkbot
Copy link
Collaborator

flinkbot commented Nov 20, 2019

CI report:

Bot commands The @flinkbot bot supports the following commands:
  • @flinkbot run travis re-run the last Travis build

@aljoscha
Copy link
Contributor

I think this is a good change! I would like to understand better what you mean by changing the hadoop.version. Do you change it in your pom? The problem only occurs when you change the hadoop.version?

@aljoscha
Copy link
Contributor

@flinkbot run travis

@aljoscha aljoscha self-assigned this Nov 21, 2019
@aljoscha aljoscha self-requested a review November 21, 2019 12:33
@liupc
Copy link
Contributor Author

liupc commented Nov 22, 2019

hi @aljoscha ,
The hadoop version of our cluster is 2.6, so I changed the hadoop.version in the flink-shaded-hadoop-2 project, and compiled it and then use some classes of it in my app. Because the xerces package was introduced into hadoop since 2.6.0(https://issues.apache.org/jira/browse/HDFS-4629), the classes of it was also packaged into my app jar, thus causing the failure.
I think if I don't depends on flink-shaded-hadoop-2 but use hadoop deps directly there are risks that this xercesImpl would be introduced.

@aljoscha
Copy link
Contributor

@flinkbot run travis

@aljoscha
Copy link
Contributor

There seems to be a test failure that looks like it can be related to the change.

@liupc
Copy link
Contributor Author

liupc commented Nov 22, 2019

@aljoscha I checked the tests failure of this build: https://travis-ci.com/flink-ci/flink/builds/137370770, it seems that it's not related with this change.

09:28:25.139 [ERROR] Failures: 
09:28:25.139 [ERROR]   HiveTableSourceTest.testProjectionPushDown:209 Sql optimization: Assertion error: type mismatch:
ref:
BIGINT
input:
VARCHAR(2147483647) CHARACTER SET "UTF-16LE"
09:28:25.139 [INFO] 
09:28:25.139 [ERROR] Tests run: 241, Failures: 1, Errors: 0, Skipped: 1

@aljoscha
Copy link
Contributor

I double checked it, the commit your PR is based on had a failure in the test. I rebased it on master and the test doesn't fail anymore. One can never be too careful. 😅

@aljoscha aljoscha merged commit fb17587 into apache:master Nov 22, 2019
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants