Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-11086] Replace flink-shaded-hadoop dependencies; add Hadoop 3 test profile #11983

Closed
wants to merge 8 commits into from

Conversation

rmetzger
Copy link
Contributor

@rmetzger rmetzger commented May 4, 2020

What is the purpose of the change

  • Replace flink-shaded-hadoop dependencies by vanilla Hadoop dependencies
    Why?
  • in principle it is better for users to use exactly the same client-hadoop versions as the server components that they are running (instead of relying on some version we are packaging for them)
  • The flink-shaded-hadoop dependencies we are providing are known to have issues. The Flink community does not have the capacity to properly maintain them. The old Hadoop2 shaded dependencies are still compatible with Flink after this change.

Open TODOs:

  • test on EMR 6, with Hadoop 3
  • Update Flink documentation

Verifying this change

I verified this change on my personal CI:

I tested the change on Google DataProc (Hadoop 2.9)

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

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

Documentation

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

@flinkbot
Copy link
Collaborator

flinkbot commented May 4, 2020

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 d63d8eb (Mon May 04 08:40:22 UTC 2020)

Warnings:

  • 23 pom.xml files were touched: Check for build and licensing issues.
  • No documentation files were touched! Remember to keep the Flink docs up to date!

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.


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 May 4, 2020

CI report:

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

@zentol zentol self-assigned this May 4, 2020
@rmetzger rmetzger marked this pull request as ready for review May 4, 2020 11:04
@rmetzger
Copy link
Contributor Author

rmetzger commented May 4, 2020

The YARN failure on Azure is a known YARN issue: https://issues.apache.org/jira/browse/FLINK-15534

flink-connectors/flink-connector-hive/pom.xml Outdated Show resolved Hide resolved
flink-connectors/flink-connector-hive/pom.xml Outdated Show resolved Hide resolved
flink-filesystems/flink-s3-fs-hadoop/pom.xml Outdated Show resolved Hide resolved
<dependencies>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs-client</artifactId>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how come this isn't required at runtime?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The artifact is required at runtime, but only if you are running Hadoop 3. Flink does only compile against 2.4.1, users need to provide the right dependencies in their classpath.
For running the tests on HD3, we provide the right classpath through this profile.

Path tmpPath = tmp.getRoot().toPath();
LOG.info("The current temporary path: {}", tmpPath);
this.sqlClientSessionConf = tmpPath.resolve("sql-client-session.conf");
this.result = tmpPath.resolve("result");

apacheAvroJars.add(downloadCache.getOrDownload("https://repo1.maven.org/maven2/org/apache/avro/avro/1.8.2/avro-1.8.2.jar", tmpPath));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So this only was only working by chance since missing stuff was provided by flink-shaded-hadoop? If so, why does that no longer work?
Related (and likely the correct fix): https://issues.apache.org/jira/browse/FLINK-17417

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So this only was only working by chance since missing stuff was provided by flink-shaded-hadoop?

Yes.

If so, why does that no longer work?

Because flink-shaded-hadoop does not exist anymore in this change

Re correct fix: I discussed this offline with @dawidwys and @twalthr and they agreed that providing the required avro dependencies is an acceptable approach.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Because flink-shaded-hadoop does not exist anymore in this change

Well duh, but hadoop is still on the classpath, no? Otherwise you're not fulfilling the contract of the Hadoop category; if the test is still passing then the category should be removed.

flink-yarn/pom.xml Outdated Show resolved Hide resolved
@rmetzger
Copy link
Contributor Author

rmetzger commented May 6, 2020

Thanks a lot for your review. I will address your comments tomorrow.

@rmetzger
Copy link
Contributor Author

rmetzger commented May 7, 2020

I addressed all comments & rebased to latest master.

Copy link
Contributor

@zentol zentol left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Java E2E tests are still missing a way to access the Hadoop classpath. This will possibly entail removing the Hadoop category and e2e-hadoop profile.

@@ -460,50 +460,6 @@ under the License.
</dependencies>
</profile>

<profile>
<!-- Copies that shaded Hadoop uber jar to the dist folder. -->
<id>include-hadoop</id>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

still referenced in the azure files and run-nightly-tests.sh

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You are right. The e2e tests will always have access to at least Hadoop 2.4.1, and we can remove the "hadoopfree" nightly profile

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Addressed in rmetzger@eda372b

@@ -153,6 +170,32 @@ under the License.

<build>
<plugins>
<plugin>
<artifactId>maven-resources-plugin</artifactId>
<!-- <version>3.1.0</version> -->
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think I copy pasted the plugin definition from somewhere and assumed that the plugin version is centrally managed.

The other uses of the plugin are also defined without a version

--
flink-quickstart/pom.xml-			<plugin>
flink-quickstart/pom.xml-				<groupId>org.apache.maven.plugins</groupId>
flink-quickstart/pom.xml:				<artifactId>maven-resources-plugin</artifactId>
flink-quickstart/pom.xml-				<configuration>
flink-quickstart/pom.xml-					<useDefaultDelimiters>false</useDefaultDelimiters>
--
flink-table/flink-sql-parser-hive/pom.xml-			<!-- adding fmpp code gen -->
flink-table/flink-sql-parser-hive/pom.xml-			<plugin>
flink-table/flink-sql-parser-hive/pom.xml:				<artifactId>maven-resources-plugin</artifactId>
flink-table/flink-sql-parser-hive/pom.xml-				<executions>
flink-table/flink-sql-parser-hive/pom.xml-					<execution>
--
flink-table/flink-sql-parser/pom.xml-			<!-- adding fmpp code gen -->
flink-table/flink-sql-parser/pom.xml-			<plugin>
flink-table/flink-sql-parser/pom.xml:				<artifactId>maven-resources-plugin</artifactId>
flink-table/flink-sql-parser/pom.xml-				<executions>
flink-table/flink-sql-parser/pom.xml-					<execution>
--
flink-walkthroughs/pom.xml-			<plugin>
flink-walkthroughs/pom.xml-				<groupId>org.apache.maven.plugins</groupId>
flink-walkthroughs/pom.xml:				<artifactId>maven-resources-plugin</artifactId>
flink-walkthroughs/pom.xml-				<configuration>
flink-walkthroughs/pom.xml-					<useDefaultDelimiters>false</useDefaultDelimiters>

It doesn't seem that we are centrally managing the plugin versions. I would propose to just remove the <!-- <version>3.1.0</version> --> line.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes we should remove it. It's a bit odd that we don't have a managed version, but that's a separate issue.

<executions>
<execution>
<id>copy-resources</id>
<!-- here the phase you need -->
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
<!-- here the phase you need -->

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry that I didn't catch this in my self-review

@@ -255,6 +298,21 @@ under the License.
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-enforcer-plugin</artifactId>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do we still need this despite excluding all dependencies?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There are convergence issues on CI for Hadoop 2.8.3 (on 2.4.1 it converges)
But since we are not providing the flink-end-to-end-tests officially as a module for users, I decided to ignore convergence:

2020-05-11T10:54:23.9920427Z [INFO] --- maven-enforcer-plugin:3.0.0-M1:enforce (dependency-convergence) @ flink-end-to-end-tests ---
2020-05-11T10:54:24.0051160Z [WARNING] 
2020-05-11T10:54:24.0051590Z Dependency convergence error for com.google.guava:guava:11.0.2 paths to dependency are:
2020-05-11T10:54:24.0052711Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0053292Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0053794Z     +-org.apache.hadoop:hadoop-common:2.8.3
2020-05-11T10:54:24.0054246Z       +-com.google.guava:guava:11.0.2
2020-05-11T10:54:24.0054558Z and
2020-05-11T10:54:24.0054967Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0055491Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0055971Z     +-org.apache.hadoop:hadoop-common:2.8.3
2020-05-11T10:54:24.0056429Z       +-org.apache.hadoop:hadoop-auth:2.8.3
2020-05-11T10:54:24.0057162Z         +-org.apache.curator:curator-framework:2.7.1
2020-05-11T10:54:24.0057681Z           +-com.google.guava:guava:16.0.1
2020-05-11T10:54:24.0057931Z and
2020-05-11T10:54:24.0058355Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0058898Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0059397Z     +-org.apache.hadoop:hadoop-common:2.8.3
2020-05-11T10:54:24.0059886Z       +-org.apache.curator:curator-client:2.7.1
2020-05-11T10:54:24.0060526Z         +-com.google.guava:guava:16.0.1
2020-05-11T10:54:24.0060915Z and
2020-05-11T10:54:24.0061396Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0061913Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0062417Z     +-org.apache.hadoop:hadoop-common:2.8.3
2020-05-11T10:54:24.0062922Z       +-org.apache.curator:curator-recipes:2.7.1
2020-05-11T10:54:24.0063396Z         +-com.google.guava:guava:16.0.1
2020-05-11T10:54:24.0063643Z and
2020-05-11T10:54:24.0064067Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0064623Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0065096Z     +-org.apache.hadoop:hadoop-hdfs:2.8.3
2020-05-11T10:54:24.0065569Z       +-com.google.guava:guava:11.0.2
2020-05-11T10:54:24.0065811Z and
2020-05-11T10:54:24.0066236Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0066774Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0067274Z     +-org.apache.hadoop:hadoop-yarn-common:2.8.3
2020-05-11T10:54:24.0067789Z       +-org.apache.hadoop:hadoop-yarn-api:2.8.3
2020-05-11T10:54:24.0068256Z         +-com.google.guava:guava:11.0.2
2020-05-11T10:54:24.0068501Z and
2020-05-11T10:54:24.0068940Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0069632Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0070133Z     +-org.apache.hadoop:hadoop-yarn-common:2.8.3
2020-05-11T10:54:24.0070582Z       +-com.google.guava:guava:11.0.2
2020-05-11T10:54:24.0071129Z and
2020-05-11T10:54:24.0071673Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0072195Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0072692Z     +-org.apache.hadoop:hadoop-yarn-client:2.8.3
2020-05-11T10:54:24.0073146Z       +-com.google.guava:guava:11.0.2
2020-05-11T10:54:24.0073347Z 
2020-05-11T10:54:24.0073499Z [WARNING] 
2020-05-11T10:54:24.0074079Z Dependency convergence error for commons-beanutils:commons-beanutils:1.8.0 paths to dependency are:
2020-05-11T10:54:24.0074699Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0075216Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0075681Z     +-org.apache.hadoop:hadoop-common:2.8.3
2020-05-11T10:54:24.0076191Z       +-commons-configuration:commons-configuration:1.7
2020-05-11T10:54:24.0076707Z         +-commons-digester:commons-digester:1.8.1
2020-05-11T10:54:24.0077208Z           +-commons-beanutils:commons-beanutils:1.8.0
2020-05-11T10:54:24.0077482Z and
2020-05-11T10:54:24.0078082Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0078619Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0079262Z     +-org.apache.hadoop:hadoop-common:2.8.3
2020-05-11T10:54:24.0079773Z       +-commons-configuration:commons-configuration:1.7
2020-05-11T10:54:24.0080296Z         +-commons-beanutils:commons-beanutils:1.8.3
2020-05-11T10:54:24.0135511Z 
2020-05-11T10:54:24.0135752Z [WARNING] 
2020-05-11T10:54:24.0136471Z Dependency convergence error for org.codehaus.jackson:jackson-mapper-asl:1.8.3 paths to dependency are:
2020-05-11T10:54:24.0137294Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0137797Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0138241Z     +-org.apache.hadoop:hadoop-common:2.8.3
2020-05-11T10:54:24.0138863Z       +-com.sun.jersey:jersey-json:1.9
2020-05-11T10:54:24.0139663Z         +-org.codehaus.jackson:jackson-mapper-asl:1.8.3
2020-05-11T10:54:24.0140120Z and
2020-05-11T10:54:24.0140860Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0141350Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0141803Z     +-org.apache.hadoop:hadoop-common:2.8.3
2020-05-11T10:54:24.0142263Z       +-org.codehaus.jackson:jackson-mapper-asl:1.9.13
2020-05-11T10:54:24.0142514Z and
2020-05-11T10:54:24.0142897Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0143382Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0143835Z     +-org.apache.hadoop:hadoop-common:2.8.3
2020-05-11T10:54:24.0144244Z       +-org.apache.avro:avro:1.8.2
2020-05-11T10:54:24.0144709Z         +-org.codehaus.jackson:jackson-mapper-asl:1.9.13
2020-05-11T10:54:24.0144946Z and
2020-05-11T10:54:24.0145341Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0145806Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0146253Z     +-org.apache.hadoop:hadoop-hdfs:2.8.3
2020-05-11T10:54:24.0146738Z       +-org.codehaus.jackson:jackson-mapper-asl:1.9.13
2020-05-11T10:54:24.0146971Z and
2020-05-11T10:54:24.0147368Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0147836Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0148302Z     +-org.apache.hadoop:hadoop-yarn-common:2.8.3
2020-05-11T10:54:24.0148766Z       +-org.codehaus.jackson:jackson-mapper-asl:1.9.13
2020-05-11T10:54:24.0148987Z 
2020-05-11T10:54:24.0149127Z [WARNING] 
2020-05-11T10:54:24.0149652Z Dependency convergence error for org.codehaus.jackson:jackson-xc:1.8.3 paths to dependency are:
2020-05-11T10:54:24.0150218Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0151586Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0152106Z     +-org.apache.hadoop:hadoop-common:2.8.3
2020-05-11T10:54:24.0152567Z       +-com.sun.jersey:jersey-json:1.9
2020-05-11T10:54:24.0153061Z         +-org.codehaus.jackson:jackson-xc:1.8.3
2020-05-11T10:54:24.0153587Z and
2020-05-11T10:54:24.0154075Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0154708Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0155178Z     +-org.apache.hadoop:hadoop-yarn-common:2.8.3
2020-05-11T10:54:24.0155638Z       +-org.codehaus.jackson:jackson-xc:1.9.13
2020-05-11T10:54:24.0155824Z 
2020-05-11T10:54:24.0155963Z [WARNING] 
2020-05-11T10:54:24.0156494Z Dependency convergence error for org.codehaus.jackson:jackson-jaxrs:1.8.3 paths to dependency are:
2020-05-11T10:54:24.0157084Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0157549Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0158003Z     +-org.apache.hadoop:hadoop-common:2.8.3
2020-05-11T10:54:24.0158413Z       +-com.sun.jersey:jersey-json:1.9
2020-05-11T10:54:24.0158869Z         +-org.codehaus.jackson:jackson-jaxrs:1.8.3
2020-05-11T10:54:24.0159109Z and
2020-05-11T10:54:24.0159494Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0159991Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0160444Z     +-org.apache.hadoop:hadoop-yarn-common:2.8.3
2020-05-11T10:54:24.0160915Z       +-org.codehaus.jackson:jackson-jaxrs:1.9.13
2020-05-11T10:54:24.0161106Z 
2020-05-11T10:54:24.0161259Z [WARNING] 
2020-05-11T10:54:24.0161975Z Dependency convergence error for org.codehaus.jackson:jackson-core-asl:1.8.3 paths to dependency are:
2020-05-11T10:54:24.0162591Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0163069Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0163535Z     +-org.apache.hadoop:hadoop-common:2.8.3
2020-05-11T10:54:24.0163977Z       +-com.sun.jersey:jersey-json:1.9
2020-05-11T10:54:24.0164608Z         +-org.codehaus.jackson:jackson-core-asl:1.8.3
2020-05-11T10:54:24.0165030Z and
2020-05-11T10:54:24.0165421Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0165933Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0166390Z     +-org.apache.hadoop:hadoop-common:2.8.3
2020-05-11T10:54:24.0166873Z       +-org.codehaus.jackson:jackson-core-asl:1.9.13
2020-05-11T10:54:24.0167125Z and
2020-05-11T10:54:24.0167516Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0168012Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0168640Z     +-org.apache.hadoop:hadoop-common:2.8.3
2020-05-11T10:54:24.0169087Z       +-org.apache.avro:avro:1.8.2
2020-05-11T10:54:24.0169556Z         +-org.codehaus.jackson:jackson-core-asl:1.9.13
2020-05-11T10:54:24.0170071Z and
2020-05-11T10:54:24.0170510Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0171026Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0171517Z     +-org.apache.hadoop:hadoop-hdfs:2.8.3
2020-05-11T10:54:24.0172151Z       +-org.codehaus.jackson:jackson-core-asl:1.9.13
2020-05-11T10:54:24.0172551Z and
2020-05-11T10:54:24.0172985Z +-org.apache.flink:flink-end-to-end-tests:1.11-SNAPSHOT
2020-05-11T10:54:24.0173498Z   +-org.apache.flink:flink-yarn_2.11:1.11-SNAPSHOT
2020-05-11T10:54:24.0173982Z     +-org.apache.hadoop:hadoop-yarn-common:2.8.3
2020-05-11T10:54:24.0174459Z       +-org.codehaus.jackson:jackson-core-asl:1.9.13
2020-05-11T10:54:24.0174662Z 

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Convergence isn't just a problem for users though; we also want convergence so the tests are stable, so i don't want to throw it out in general just because Hadoop is causing yet another issue.
I'd also like to understand why the exclusions aren't working; we used them in other areas to solve convergence issues IIRC.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

But I don't know if it makes sense to mess with Hadoop's transitive dependency tree.
In our tests, we want to ensure that Flink works with certain vanilla Hadoop versions.

If we start hand-crafting Hadoop's dependencies towards convergence, we won't ensure that Flink works with those versions -- we ensure it works with our version.

If the maven-enforcer-plugin would allow us to control the convergence check more fine-grained, I would not be opposed to it, as we need to ensure that some dependencyManagement, exclusion etc. from us is affecting vanilla Hadoop's dependency tree.

A second problem is that the exclusions might differ between the Hadoop versions we use for CI. For Hadoop 2.4.1 we have convergence, for 2.8.3 we don't.

Given these thoughts, I believe we should disable the convergence check for the tests, and rely on test failures for detecting severe issues with our Hadoop integration. We just need to accept that the Hadoop project is wild-west when it comes to dependencies.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not suggesting to converge hadoop dependencies, that is utterly futile.

What I'm suggesting is to ensure that the enforcer ignores flink-yarn but continues to run for other sub-modules, which it should because of the exclusions. Hence, it is important to understand why the exclusions aren't working here when they are working in other places.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Okay, now I understand. I will look into it tomorrow.

"-t", flinkLibFolder.getAbsolutePath(),
"-t", flinkShadedHadoopDir.getAbsolutePath(),
"-jm", "768m",
"-t", flinkLibFolder.getAbsolutePath(), "-jm", "768m",
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
"-t", flinkLibFolder.getAbsolutePath(), "-jm", "768m",
"-t", flinkLibFolder.getAbsolutePath(),
"-jm", "768m",

@@ -93,9 +95,12 @@
private Path result;
private Path sqlClientSessionConf;

private static final DownloadCache downloadCache = DownloadCache.get();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

add @ClassRule

@@ -142,6 +144,11 @@ private static Configuration initialize(Configuration conf) {

@BeforeClass
public static void setUp() throws Exception {
// HBase 1.4 does not work with Hadoop 3
// because it uses Guava 12.0.1, Hadoop 3 uses Guava 27.0-jre.
// There is not Guava version in between that works with both.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
// There is not Guava version in between that works with both.
// There is no Guava version in between that works with both.

@@ -120,4 +88,44 @@ This way it should work both in local and cluster run where the provided depende
To run or debug an application in IntelliJ Idea the provided dependencies can be included to the class path
in the "Run|Edit Configurations" window.


2) Putting the required jar files into /lib directory of the Flink distribution
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This paragraph isn't properly integrated with the current documentation. I guess it should be removed since the section below subsumes it?




The Flink project used to release Hadoop distributions for specific versions, that relocate or exclude several dependencies
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

add a specific release as a date, instead of "used to"

@rmetzger
Copy link
Contributor Author

Thanks a lot for your second big review! I addressed all your comments and kicked of a custom CI build in my personal Azure account that runs all the nightly builds, to make sure we are not breaking anything in any other Hadoop version from the review changes: https://dev.azure.com/rmetzger/Flink/_build/results?buildId=7975&view=results

@rmetzger
Copy link
Contributor Author

Note to self: close this JIRA when merging this PR: https://issues.apache.org/jira/browse/FLINK-17563

@rmetzger
Copy link
Contributor Author

I pushed a new commit addressing the dependency convergence issue you've raised.

@@ -120,4 +88,44 @@ This way it should work both in local and cluster run where the provided depende
To run or debug an application in IntelliJ Idea the provided dependencies can be included to the class path
in the "Run|Edit Configurations" window.


2) Putting the required jar files into /lib directory of the Flink distribution
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

chinese version wasn't updated

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will do that before merging or with the next batch of review comments.

Copy link
Contributor

@zentol zentol left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Some smaller things, otherwise +1.

<groupId>org.apache.flink</groupId>
<artifactId>flink-shaded-hadoop-2</artifactId>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-common</artifactId>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Remove the hbase hadoop exclusions instead

@@ -97,8 +88,8 @@ under the License.

<!-- Replacement for ORC's Hadoop dependency. -->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-shaded-hadoop-2</artifactId>
<groupId>org.apache.hadoop</groupId>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

redundant, pulled in by orc-core

@@ -63,7 +63,7 @@ under the License.
<version>${orc.version}</version>
<classifier>nohive</classifier>
<exclusions>
<!-- Exclude ORC's Hadoop dependency and pull in Flink's shaded Hadoop. -->
<!-- Exclude ORC's Hadoop dependency and pull in Flink's Hadoop. -->
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

let's remove the exclusions just like in flink-orc, and then the explicit hadoop dependencies

@rmetzger
Copy link
Contributor Author

Thanks a lot for your reviews. I have rebased to the latest master, resolved some conflicts and addressed your remaining comments.
Once (my personal) CI is green, I will merge this change.

@rmetzger rmetzger closed this in 47323a4 May 13, 2020
jnh5y pushed a commit to jnh5y/flink that referenced this pull request Dec 18, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
3 participants