You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by rm...@apache.org on 2020/05/13 14:28:54 UTC

[flink] branch master updated (6a6a439 -> 47323a4)

This is an automated email from the ASF dual-hosted git repository.

rmetzger pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git.


    from 6a6a439  [FLINK-17537][jdbc] Refactor flink-jdbc connector structure
     new e1e7d7f  [FLINK-11086] Replace flink-shaded-hadoop-2 dependency by vanilla Hadoop dependency
     new 07330dd  [FLINK-11086][e2e] Properly add Avro JARs in SQLClientKafkaITCase
     new 90fbb94  [FLINK-11086][HBase] Skip some HBase tests in Hadoop 3
     new 082061d  [FLINK-11086][yarn] Use YARN_APPLICATION_CLASSPATH instead of flink-shaded-hadoop fat jar in tests
     new c3648da  [FLINK-11086][e2e] Use HADOOP_CLASSPATH in end to end tests
     new 75c9cf1  [FLINK-11086][AZP] Add Hadoop3 test profile to nightlies
     new 2cc63a6  [FLINK-11086][docs] Make HADOOP_CLASSPATH approach more prominent in docs
     new 47323a4  [FLINK-11086] Clean up profiles and dependency exclusions

The 8 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 azure-pipelines.yml                                |   2 +-
 docs/ops/deployment/hadoop.md                      |  81 ++++++++---------
 docs/ops/deployment/hadoop.zh.md                   |  81 ++++++++---------
 docs/ops/deployment/yarn_setup.md                  |  23 ++---
 docs/ops/deployment/yarn_setup.zh.md               |  27 +++---
 .../flink-connector-filesystem/pom.xml             |  16 +++-
 flink-connectors/flink-connector-hive/pom.xml      |  95 +++++++++++++++++++-
 .../flink-hadoop-compatibility/pom.xml             |  10 ++-
 flink-connectors/flink-hbase/pom.xml               |  43 ++-------
 .../hbase/util/HBaseTestingClusterAutoStarter.java |  11 +++
 flink-connectors/flink-hcatalog/pom.xml            |  10 ++-
 flink-dist/pom.xml                                 |  48 +---------
 flink-dist/src/main/assemblies/hadoop.xml          |  41 ---------
 .../flink-bucketing-sink-test/pom.xml              |  31 ++++++-
 .../tests/util/kafka/SQLClientKafkaITCase.java     |  20 ++++-
 .../flink-end-to-end-tests-common/pom.xml          |  51 +++++++++++
 .../apache/flink/tests/util/categories/Hadoop.java |  25 ------
 .../flink/tests/util/flink/SQLJobSubmission.java   |   5 ++
 flink-end-to-end-tests/pom.xml                     |  48 ++++++++--
 flink-end-to-end-tests/run-nightly-tests.sh        |   8 +-
 flink-end-to-end-tests/test-scripts/common.sh      |  10 +++
 .../test-scripts/common_mesos_docker.sh            |   8 +-
 .../docker-mesos-cluster/docker-compose.yml        |   1 +
 .../test_mesos_multiple_submissions.sh             |   4 +-
 .../test-scripts/test_mesos_wordcount.sh           |   4 +-
 .../test-scripts/test_streaming_bucketing.sh       |   2 +
 flink-filesystems/flink-hadoop-fs/pom.xml          |  17 +++-
 flink-filesystems/flink-mapr-fs/pom.xml            |  10 ++-
 flink-filesystems/flink-s3-fs-hadoop/pom.xml       |  11 +++
 flink-filesystems/flink-swift-fs-hadoop/pom.xml    |  24 ++++-
 flink-formats/flink-compress/pom.xml               |  10 ++-
 flink-formats/flink-orc-nohive/pom.xml             |  35 ++++----
 flink-formats/flink-orc/pom.xml                    |  33 +++----
 flink-formats/flink-parquet/pom.xml                |  10 ++-
 flink-formats/flink-sequence-file/pom.xml          |   9 +-
 flink-fs-tests/pom.xml                             |  26 +++++-
 flink-runtime/pom.xml                              |  16 +++-
 flink-tests/pom.xml                                |   8 +-
 flink-yarn-tests/pom.xml                           | 100 ++++++++++-----------
 .../flink/yarn/YARNFileReplicationITCase.java      |   1 -
 .../flink/yarn/YARNHighAvailabilityITCase.java     |   5 --
 .../yarn/YARNSessionCapacitySchedulerITCase.java   |   7 --
 .../apache/flink/yarn/YARNSessionFIFOITCase.java   |   5 --
 .../apache/flink/yarn/YarnConfigurationITCase.java |   1 -
 .../flink/yarn/YarnPrioritySchedulingITCase.java   |   1 -
 .../java/org/apache/flink/yarn/YarnTestBase.java   |  25 ++++--
 flink-yarn/pom.xml                                 |  25 +++++-
 .../flink/yarn/YarnFileStageTestS3ITCase.java      |   5 ++
 pom.xml                                            |  61 +++++++++++--
 tools/azure-pipelines/build-apache-repo.yml        |  22 ++---
 tools/verify_scala_suffixes.sh                     |   2 +-
 51 files changed, 725 insertions(+), 449 deletions(-)
 delete mode 100644 flink-dist/src/main/assemblies/hadoop.xml
 delete mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/Hadoop.java


[flink] 03/08: [FLINK-11086][HBase] Skip some HBase tests in Hadoop 3

Posted by rm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rmetzger pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 90fbb9447f4e59ac3bf7c313bbc0c7274cb0d907
Author: Robert Metzger <rm...@apache.org>
AuthorDate: Mon May 4 10:12:53 2020 +0200

    [FLINK-11086][HBase] Skip some HBase tests in Hadoop 3
---
 .../addons/hbase/util/HBaseTestingClusterAutoStarter.java     | 11 +++++++++++
 1 file changed, 11 insertions(+)

diff --git a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/util/HBaseTestingClusterAutoStarter.java b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/util/HBaseTestingClusterAutoStarter.java
index b515c2b..0fe8ad4 100644
--- a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/util/HBaseTestingClusterAutoStarter.java
+++ b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/util/HBaseTestingClusterAutoStarter.java
@@ -35,7 +35,9 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.util.VersionUtil;
 import org.junit.AfterClass;
+import org.junit.Assume;
 import org.junit.BeforeClass;
 
 import java.io.File;
@@ -142,6 +144,11 @@ public abstract class HBaseTestingClusterAutoStarter extends AbstractTestBase {
 
 	@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.
+		Assume.assumeTrue("This test is skipped for Hadoop versions above 3", VersionUtil.compareVersions(System.getProperty("hadoop.version"), "3.0.0") < 0);
+
 		LOG.info("HBase minicluster: Starting");
 
 		TEST_UTIL.startMiniCluster(1);
@@ -236,6 +243,10 @@ public abstract class HBaseTestingClusterAutoStarter extends AbstractTestBase {
 
 	@AfterClass
 	public static void tearDown() throws Exception {
+		if (conf == null) {
+			LOG.info("Skipping Hbase tear down. It was never started");
+			return;
+		}
 		LOG.info("HBase minicluster: Shutting down");
 		deleteTables();
 		hbaseSiteXmlFile.delete();


[flink] 01/08: [FLINK-11086] Replace flink-shaded-hadoop-2 dependency by vanilla Hadoop dependency

Posted by rm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rmetzger pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit e1e7d7f7ecc080c850a264021bf1b20e3d27d373
Author: Robert Metzger <rm...@apache.org>
AuthorDate: Mon May 4 10:07:17 2020 +0200

    [FLINK-11086] Replace flink-shaded-hadoop-2 dependency by vanilla Hadoop dependency
---
 .../flink-connector-filesystem/pom.xml             | 16 +++-
 flink-connectors/flink-connector-hive/pom.xml      | 95 +++++++++++++++++++++-
 .../flink-hadoop-compatibility/pom.xml             | 10 ++-
 flink-connectors/flink-hbase/pom.xml               |  4 +-
 flink-connectors/flink-hcatalog/pom.xml            | 10 ++-
 flink-dist/pom.xml                                 | 48 +----------
 flink-dist/src/main/assemblies/hadoop.xml          | 41 ----------
 .../flink-bucketing-sink-test/pom.xml              |  9 +-
 flink-end-to-end-tests/pom.xml                     | 15 ++++
 flink-filesystems/flink-hadoop-fs/pom.xml          | 17 +++-
 flink-filesystems/flink-mapr-fs/pom.xml            | 10 ++-
 flink-filesystems/flink-s3-fs-hadoop/pom.xml       | 11 +++
 flink-filesystems/flink-swift-fs-hadoop/pom.xml    | 24 +++++-
 flink-formats/flink-compress/pom.xml               | 10 ++-
 flink-formats/flink-orc-nohive/pom.xml             | 29 ++++++-
 flink-formats/flink-orc/pom.xml                    | 30 ++++---
 flink-formats/flink-parquet/pom.xml                | 10 ++-
 flink-formats/flink-sequence-file/pom.xml          |  9 +-
 flink-fs-tests/pom.xml                             | 26 +++++-
 flink-runtime/pom.xml                              | 16 +++-
 flink-tests/pom.xml                                |  8 +-
 flink-yarn-tests/pom.xml                           | 24 ------
 flink-yarn/pom.xml                                 | 25 +++++-
 pom.xml                                            | 61 ++++++++++++--
 24 files changed, 390 insertions(+), 168 deletions(-)

diff --git a/flink-connectors/flink-connector-filesystem/pom.xml b/flink-connectors/flink-connector-filesystem/pom.xml
index 5a7a94e..5b6489a 100644
--- a/flink-connectors/flink-connector-filesystem/pom.xml
+++ b/flink-connectors/flink-connector-filesystem/pom.xml
@@ -51,8 +51,20 @@ under the License.
 		</dependency>
 
 		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-hadoop-2</artifactId>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-hdfs</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-mapreduce-client-core</artifactId>
 			<scope>provided</scope>
 		</dependency>
 
diff --git a/flink-connectors/flink-connector-hive/pom.xml b/flink-connectors/flink-connector-hive/pom.xml
index 48ae9edd..9dd9426 100644
--- a/flink-connectors/flink-connector-hive/pom.xml
+++ b/flink-connectors/flink-connector-hive/pom.xml
@@ -41,6 +41,91 @@ under the License.
 		<derby.version>10.10.2.0</derby.version>
 	</properties>
 
+	<!-- Overwrite hadoop dependency management from flink-parent to use locally defined Hadoop version -->
+	<dependencyManagement>
+		<dependencies>
+			<dependency>
+				<groupId>org.apache.hadoop</groupId>
+				<artifactId>hadoop-common</artifactId>
+				<version>${hivemetastore.hadoop.version}</version>
+				<exclusions>
+					<exclusion>
+						<groupId>log4j</groupId>
+						<artifactId>log4j</artifactId>
+					</exclusion>
+					<exclusion>
+						<groupId>org.slf4j</groupId>
+						<artifactId>slf4j-log4j12</artifactId>
+					</exclusion>
+				</exclusions>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.hadoop</groupId>
+				<artifactId>hadoop-hdfs</artifactId>
+				<version>${hivemetastore.hadoop.version}</version>
+				<exclusions>
+					<exclusion>
+						<groupId>log4j</groupId>
+						<artifactId>log4j</artifactId>
+					</exclusion>
+					<exclusion>
+						<groupId>org.slf4j</groupId>
+						<artifactId>slf4j-log4j12</artifactId>
+					</exclusion>
+				</exclusions>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.hadoop</groupId>
+				<artifactId>hadoop-mapreduce-client-core</artifactId>
+				<version>${hivemetastore.hadoop.version}</version>
+				<exclusions>
+					<exclusion>
+						<groupId>log4j</groupId>
+						<artifactId>log4j</artifactId>
+					</exclusion>
+					<exclusion>
+						<groupId>org.slf4j</groupId>
+						<artifactId>slf4j-log4j12</artifactId>
+					</exclusion>
+				</exclusions>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.hadoop</groupId>
+				<artifactId>hadoop-yarn-common</artifactId>
+				<version>${hivemetastore.hadoop.version}</version>
+				<exclusions>
+					<exclusion>
+						<groupId>log4j</groupId>
+						<artifactId>log4j</artifactId>
+					</exclusion>
+					<exclusion>
+						<groupId>org.slf4j</groupId>
+						<artifactId>slf4j-log4j12</artifactId>
+					</exclusion>
+				</exclusions>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.hadoop</groupId>
+				<artifactId>hadoop-yarn-client</artifactId>
+				<version>${hivemetastore.hadoop.version}</version>
+				<exclusions>
+					<exclusion>
+						<groupId>log4j</groupId>
+						<artifactId>log4j</artifactId>
+					</exclusion>
+					<exclusion>
+						<groupId>org.slf4j</groupId>
+						<artifactId>slf4j-log4j12</artifactId>
+					</exclusion>
+				</exclusions>
+			</dependency>
+		</dependencies>
+	</dependencyManagement>
+
 	<dependencies>
 
 		<!-- core dependencies -->
@@ -126,12 +211,16 @@ under the License.
 			thus override the default hadoop version from 2.4.1 to 2.7.5
 		-->
 		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-hadoop-2-uber</artifactId>
-			<version>${hivemetastore.hadoop.version}-${flink.shaded.version}</version>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
 			<scope>provided</scope>
 		</dependency>
 
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-mapreduce-client-core</artifactId>
+			<scope>provided</scope>
+		</dependency>
 		<!-- Hive dependencies -->
 		<!-- Note: Hive published jars do not have proper dependencies declared.
 		We need to push for HIVE-16391 (https://issues.apache.org/jira/browse/HIVE-16391) to resolve this problem. -->
diff --git a/flink-connectors/flink-hadoop-compatibility/pom.xml b/flink-connectors/flink-hadoop-compatibility/pom.xml
index 46237dd..efcc96c 100644
--- a/flink-connectors/flink-hadoop-compatibility/pom.xml
+++ b/flink-connectors/flink-hadoop-compatibility/pom.xml
@@ -61,8 +61,14 @@ under the License.
 		</dependency>
 
 		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-hadoop-2</artifactId>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-mapreduce-client-core</artifactId>
 			<scope>provided</scope>
 		</dependency>
 
diff --git a/flink-connectors/flink-hbase/pom.xml b/flink-connectors/flink-hbase/pom.xml
index 6c46bbd..cd31cfb 100644
--- a/flink-connectors/flink-hbase/pom.xml
+++ b/flink-connectors/flink-hbase/pom.xml
@@ -101,8 +101,8 @@ under the License.
 		</dependency>
 
 		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-hadoop-2</artifactId>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
 			<scope>provided</scope>
 		</dependency>
 
diff --git a/flink-connectors/flink-hcatalog/pom.xml b/flink-connectors/flink-hcatalog/pom.xml
index 1eb6cc4..fc84696 100644
--- a/flink-connectors/flink-hcatalog/pom.xml
+++ b/flink-connectors/flink-hcatalog/pom.xml
@@ -82,8 +82,14 @@ under the License.
 		</dependency>
 
 		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-hadoop-2</artifactId>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-mapreduce-client-core</artifactId>
 			<scope>provided</scope>
 		</dependency>
 
diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml
index 3225fa0..59ada96 100644
--- a/flink-dist/pom.xml
+++ b/flink-dist/pom.xml
@@ -137,8 +137,8 @@ under the License.
 			<version>${project.version}</version>
 			<exclusions>
 				<exclusion>
-					<groupId>org.apache.flink</groupId>
-					<artifactId>flink-shaded-hadoop-2</artifactId>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>*</artifactId>
 				</exclusion>
 			</exclusions>
 		</dependency>
@@ -461,50 +461,6 @@ under the License.
 		</profile>
 
 		<profile>
-			<!-- Copies that shaded Hadoop uber jar to the dist folder. -->
-			<id>include-hadoop</id>
-			<activation>
-				<property>
-					<name>include-hadoop</name>
-				</property>
-			</activation>
-			<dependencies>
-				<!--
-                    The Hadoop 2 Uber jar should not go into the Flink dist jar, but
-                    sit next to it. Hence, we set it to 'provided' here.
-                -->
-				<dependency>
-					<groupId>org.apache.flink</groupId>
-					<artifactId>flink-shaded-hadoop-2-uber</artifactId>
-					<scope>provided</scope>
-				</dependency>
-			</dependencies>
-			<build>
-				<plugins>
-					<plugin>
-						<artifactId>maven-assembly-plugin</artifactId>
-						<executions>
-							<execution>
-								<id>hadoop</id>
-								<phase>package</phase>
-								<goals>
-									<goal>single</goal>
-								</goals>
-								<configuration>
-									<descriptors>
-										<descriptor>src/main/assemblies/hadoop.xml</descriptor>
-									</descriptors>
-									<finalName>flink-${project.version}-bin</finalName>
-									<appendAssemblyId>false</appendAssemblyId>
-								</configuration>
-							</execution>
-						</executions>
-					</plugin>
-
-				</plugins>
-			</build>
-		</profile>
-		<profile>
 			<!-- Creates/Removes the 'build-target' symlink in the root directory (only Unix systems) -->
 			<id>symlink-build-target</id>
 			<activation>
diff --git a/flink-dist/src/main/assemblies/hadoop.xml b/flink-dist/src/main/assemblies/hadoop.xml
deleted file mode 100644
index 7a3ebc4..0000000
--- a/flink-dist/src/main/assemblies/hadoop.xml
+++ /dev/null
@@ -1,41 +0,0 @@
-<!--
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-  http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing,
-software distributed under the License is distributed on an
-"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-KIND, either express or implied.  See the License for the
-specific language governing permissions and limitations
-under the License.
--->
-<assembly
-	xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0"
-	xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-	xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0 http://maven.apache.org/xsd/assembly-1.1.0.xsd">
-	<id>hadoop</id>
-	<formats>
-		<format>dir</format>
-	</formats>
-
-	<includeBaseDirectory>true</includeBaseDirectory>
-	<baseDirectory>flink-${project.version}</baseDirectory>
-
-	<dependencySets>
-		<dependencySet>
-			<outputDirectory>lib/</outputDirectory>
-			<useTransitiveDependencies>true</useTransitiveDependencies>
-			<scope>provided</scope>
-			<includes>
-				<include>org.apache.flink:flink-shaded-hadoop-2-uber</include>
-			</includes>
-		</dependencySet>
-	</dependencySets>
-</assembly>
diff --git a/flink-end-to-end-tests/flink-bucketing-sink-test/pom.xml b/flink-end-to-end-tests/flink-bucketing-sink-test/pom.xml
index 7e010d9..433215c 100644
--- a/flink-end-to-end-tests/flink-bucketing-sink-test/pom.xml
+++ b/flink-end-to-end-tests/flink-bucketing-sink-test/pom.xml
@@ -47,8 +47,13 @@
 			<version>${project.version}</version>
 		</dependency>
 		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-hadoop-2</artifactId>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
+			<scope>provided</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-hdfs</artifactId>
 			<scope>provided</scope>
 			<exclusions>
 				<!-- Needed for proper dependency convergence -->
diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml
index 8689fae..3b97256 100644
--- a/flink-end-to-end-tests/pom.xml
+++ b/flink-end-to-end-tests/pom.xml
@@ -255,6 +255,21 @@ under the License.
 					</execution>
 				</executions>
 			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-enforcer-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>dependency-convergence</id>
+						<goals>
+							<goal>enforce</goal>
+						</goals>
+						<configuration>
+							<skip>true</skip>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
 		</plugins>
 
 		<pluginManagement>
diff --git a/flink-filesystems/flink-hadoop-fs/pom.xml b/flink-filesystems/flink-hadoop-fs/pom.xml
index f5c247e..a8f07c6 100644
--- a/flink-filesystems/flink-hadoop-fs/pom.xml
+++ b/flink-filesystems/flink-hadoop-fs/pom.xml
@@ -43,10 +43,23 @@ under the License.
 		<!-- pulling in Hadoop by default -->
 
 		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-hadoop-2</artifactId>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
+			<optional>true</optional>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-hdfs</artifactId>
+			<optional>true</optional>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-mapreduce-client-core</artifactId>
 			<optional>true</optional>
 		</dependency>
+		
 
 		<!-- for the behavior test suite -->
 		<dependency>
diff --git a/flink-filesystems/flink-mapr-fs/pom.xml b/flink-filesystems/flink-mapr-fs/pom.xml
index 32725fb..1d914ea 100644
--- a/flink-filesystems/flink-mapr-fs/pom.xml
+++ b/flink-filesystems/flink-mapr-fs/pom.xml
@@ -47,8 +47,14 @@ under the License.
 		</dependency>
 
 		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-hadoop-2</artifactId>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
+			<optional>true</optional>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-hdfs</artifactId>
 			<optional>true</optional>
 		</dependency>
 
diff --git a/flink-filesystems/flink-s3-fs-hadoop/pom.xml b/flink-filesystems/flink-s3-fs-hadoop/pom.xml
index 58c5b12..db569a4 100644
--- a/flink-filesystems/flink-s3-fs-hadoop/pom.xml
+++ b/flink-filesystems/flink-s3-fs-hadoop/pom.xml
@@ -32,6 +32,17 @@ under the License.
 
 	<packaging>jar</packaging>
 
+	<!-- Override the flink-parent dependencyManagement definition for hadoop-common to ensure
+		${fs.hadoopshaded.version} is used for this file system -->
+	<dependencyManagement>
+		<dependencies>
+			<dependency>
+				<groupId>org.apache.hadoop</groupId>
+				<artifactId>hadoop-common</artifactId>
+				<version>${fs.hadoopshaded.version}</version>
+			</dependency>
+		</dependencies>
+	</dependencyManagement>
 	<dependencies>
 
 		<!-- Flink's file system abstraction (compiled against, not bundled) -->
diff --git a/flink-filesystems/flink-swift-fs-hadoop/pom.xml b/flink-filesystems/flink-swift-fs-hadoop/pom.xml
index 2307a1e..b63a068 100644
--- a/flink-filesystems/flink-swift-fs-hadoop/pom.xml
+++ b/flink-filesystems/flink-swift-fs-hadoop/pom.xml
@@ -37,6 +37,26 @@ under the License.
 		<openstackhadoop.hadoop.version>2.8.1</openstackhadoop.hadoop.version>
 	</properties>
 
+	<!-- Overwrite hadoop dependency versions inherited from the parent pom dependencyManagement -->
+	<dependencyManagement>
+		<dependencies>
+			<dependency>
+				<groupId>org.apache.hadoop</groupId>
+				<artifactId>hadoop-client</artifactId>
+				<version>${openstackhadoop.hadoop.version}</version>
+			</dependency>
+			<dependency>
+				<groupId>org.apache.hadoop</groupId>
+				<artifactId>hadoop-common</artifactId>
+				<version>${openstackhadoop.hadoop.version}</version>
+			</dependency>
+			<dependency>
+				<groupId>org.apache.hadoop</groupId>
+				<artifactId>hadoop-hdfs</artifactId>
+				<version>${openstackhadoop.hadoop.version}</version>
+			</dependency>
+		</dependencies>
+	</dependencyManagement>
 	<dependencies>
 
 		<!-- Flink core -->
@@ -59,8 +79,8 @@ under the License.
 				because the optional Hadoop dependency is also pulled in for tests -->
 			<exclusions>
 				<exclusion>
-					<groupId>org.apache.flink</groupId>
-					<artifactId>flink-shaded-hadoop-2</artifactId>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-hdfs</artifactId>
 				</exclusion>
 			</exclusions>
 		</dependency>
diff --git a/flink-formats/flink-compress/pom.xml b/flink-formats/flink-compress/pom.xml
index 6446944..84a73aa 100644
--- a/flink-formats/flink-compress/pom.xml
+++ b/flink-formats/flink-compress/pom.xml
@@ -44,8 +44,14 @@ under the License.
 		</dependency>
 
 		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-hadoop-2</artifactId>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-hdfs</artifactId>
 			<scope>provided</scope>
 		</dependency>
 
diff --git a/flink-formats/flink-orc-nohive/pom.xml b/flink-formats/flink-orc-nohive/pom.xml
index 3ef6cec..f352313 100644
--- a/flink-formats/flink-orc-nohive/pom.xml
+++ b/flink-formats/flink-orc-nohive/pom.xml
@@ -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. -->
 				<exclusion>
 					<groupId>org.apache.hadoop</groupId>
 					<artifactId>hadoop-common</artifactId>
@@ -81,12 +81,17 @@ under the License.
 
 		<!-- Replacement for ORC's Hadoop dependency. -->
 		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-hadoop-2</artifactId>
-			<version>${hadoop.version}-${flink.shaded.version}</version>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
 			<scope>provided</scope>
 		</dependency>
 
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-hdfs</artifactId>
+			<scope>provided</scope>
+		</dependency>
+		
 		<!-- test dependencies -->
 		<dependency>
 			<groupId>org.apache.flink</groupId>
@@ -98,6 +103,22 @@ under the License.
 
 	</dependencies>
 
+	<profiles>
+		<profile>
+			<!-- This profile adds dependencies needed to execute the tests
+			with Hadoop 3 -->
+			<id>hadoop3-tests</id>
+			<dependencies>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-hdfs-client</artifactId>
+					<version>${hadoop.version}</version>
+					<scope>test</scope>
+				</dependency>
+			</dependencies>
+		</profile>
+	</profiles>
+
 	<build>
 		<plugins>
 			<!-- skip dependency convergence due to Hadoop dependency -->
diff --git a/flink-formats/flink-orc/pom.xml b/flink-formats/flink-orc/pom.xml
index 3aa36cb..00a39e2 100644
--- a/flink-formats/flink-orc/pom.xml
+++ b/flink-formats/flink-orc/pom.xml
@@ -75,15 +75,6 @@ under the License.
 			<artifactId>orc-core</artifactId>
 			<version>${orc.version}</version>
 			<exclusions>
-				<!-- Exclude ORC's Hadoop dependency and pull in Flink's shaded Hadoop. -->
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-common</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-hdfs</artifactId>
-				</exclusion>
 				<exclusion>
 					<groupId>javax.activation</groupId>
 					<artifactId>javax.activation-api</artifactId>
@@ -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>
+			<artifactId>hadoop-common</artifactId>
 			<scope>provided</scope>
 		</dependency>
 
@@ -147,6 +138,23 @@ under the License.
 
 	</dependencies>
 
+	<profiles>
+		<profile>
+			<!-- This profile adds dependencies needed to execute the tests
+			with Hadoop 3 -->
+			<id>hadoop3-tests</id>
+			<dependencies>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-hdfs-client</artifactId>
+					<version>${hadoop.version}</version>
+					<scope>test</scope>
+				</dependency>
+			</dependencies>
+		</profile>
+	</profiles>
+
+
 	<build>
 		<plugins>
 			<!-- skip dependency convergence due to Hadoop dependency -->
diff --git a/flink-formats/flink-parquet/pom.xml b/flink-formats/flink-parquet/pom.xml
index 00b1d3e..1ca05fb 100644
--- a/flink-formats/flink-parquet/pom.xml
+++ b/flink-formats/flink-parquet/pom.xml
@@ -97,8 +97,14 @@ under the License.
 
 		<!-- Hadoop is needed by Parquet -->
 		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-hadoop-2</artifactId>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-mapreduce-client-core</artifactId>
 			<scope>provided</scope>
 		</dependency>
 
diff --git a/flink-formats/flink-sequence-file/pom.xml b/flink-formats/flink-sequence-file/pom.xml
index 2d0bc88..9f99598 100644
--- a/flink-formats/flink-sequence-file/pom.xml
+++ b/flink-formats/flink-sequence-file/pom.xml
@@ -45,11 +45,16 @@ under the License.
 
 		<!-- Hadoop is needed for SequenceFile -->
 		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-hadoop-2</artifactId>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
 			<scope>provided</scope>
 		</dependency>
 
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-hdfs</artifactId>
+			<scope>provided</scope>
+		</dependency>
 		<!-- test dependencies -->
 
 		<dependency>
diff --git a/flink-fs-tests/pom.xml b/flink-fs-tests/pom.xml
index da60966..80230b9 100644
--- a/flink-fs-tests/pom.xml
+++ b/flink-fs-tests/pom.xml
@@ -37,8 +37,14 @@ under the License.
 	-->
 	<dependencies>
 		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-hadoop-2</artifactId>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-hdfs</artifactId>
 			<scope>test</scope>
 		</dependency>
 		
@@ -130,6 +136,22 @@ under the License.
 		</dependency>
 	</dependencies>
 
+	<profiles>
+		<profile>
+			<!-- This profile adds dependencies needed to execute the tests
+			with Hadoop 3 -->
+			<id>hadoop3-tests</id>
+			<dependencies>
+				<dependency>
+					<groupId>org.apache.hadoop</groupId>
+					<artifactId>hadoop-hdfs-client</artifactId>
+					<version>${hadoop.version}</version>
+					<scope>test</scope>
+				</dependency>
+			</dependencies>
+		</profile>
+	</profiles>
+
 	<build>
 		<plugins>
 			<plugin>
diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml
index f4366a6..5c51e73 100644
--- a/flink-runtime/pom.xml
+++ b/flink-runtime/pom.xml
@@ -68,8 +68,20 @@ under the License.
 		<!-- optional dependency on Hadoop, so that Hadoop classes are not always pulled in -->
 
 		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-hadoop-2</artifactId>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
+			<optional>true</optional>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-hdfs</artifactId>
+			<optional>true</optional>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-mapreduce-client-core</artifactId>
 			<optional>true</optional>
 		</dependency>
 
diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml
index b62c8d6..53f781c 100644
--- a/flink-tests/pom.xml
+++ b/flink-tests/pom.xml
@@ -58,12 +58,10 @@ under the License.
 			<scope>test</scope>
 		</dependency>
 
-		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-hadoop-2</artifactId>
+		<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
 			<scope>test</scope>
-		</dependency>
-
+</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-shaded-jackson</artifactId>
diff --git a/flink-yarn-tests/pom.xml b/flink-yarn-tests/pom.xml
index 0e3b932..2f5e9f4 100644
--- a/flink-yarn-tests/pom.xml
+++ b/flink-yarn-tests/pom.xml
@@ -90,14 +90,6 @@ under the License.
 			<artifactId>flink-yarn_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<!-- prevent flink-shaded-hadoop from being on the test classpath
-					 	to avoid conflicts with other dependencies -->
-					<groupId>org.apache.flink</groupId>
-					<artifactId>flink-shaded-hadoop-2</artifactId>
-				</exclusion>
-			</exclusions>
 		</dependency>
 
 		<dependency>
@@ -106,14 +98,6 @@ under the License.
 			<version>${project.version}</version>
 			<type>test-jar</type>
 			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<!-- prevent flink-shaded-hadoop from being on the test classpath
-					 	to avoid conflicts with other dependencies -->
-					<groupId>org.apache.flink</groupId>
-					<artifactId>flink-shaded-hadoop-2</artifactId>
-				</exclusion>
-			</exclusions>
 		</dependency>
 
 		<dependency>
@@ -401,14 +385,6 @@ under the License.
 					<artifactItems>
 						<artifactItem>
 							<groupId>org.apache.flink</groupId>
-							<artifactId>flink-shaded-hadoop-2-uber</artifactId>
-							<version>${hadoop.version}-${flink.shaded.version}</version>
-							<type>jar</type>
-							<overWrite>true</overWrite>
-							<outputDirectory>${project.build.directory}/shaded-hadoop</outputDirectory>
-						</artifactItem>
-						<artifactItem>
-							<groupId>org.apache.flink</groupId>
 							<artifactId>flink-examples-batch_${scala.binary.version}</artifactId>
 							<type>jar</type>
 							<classifier>WordCount</classifier>
diff --git a/flink-yarn/pom.xml b/flink-yarn/pom.xml
index a9bec80..ac35e6b 100644
--- a/flink-yarn/pom.xml
+++ b/flink-yarn/pom.xml
@@ -54,9 +54,30 @@ under the License.
 		</dependency>
 
 		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-shaded-hadoop-2</artifactId>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-hdfs</artifactId>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-yarn-common</artifactId>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-yarn-client</artifactId>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-mapreduce-client-core</artifactId>
 		</dependency>
+		
 
 		<!-- test dependencies -->
 
diff --git a/pom.xml b/pom.xml
index 0599b4b..9d47193 100644
--- a/pom.xml
+++ b/pom.xml
@@ -308,9 +308,9 @@ under the License.
 			</dependency>
 
 			<dependency>
-				<groupId>org.apache.flink</groupId>
-				<artifactId>flink-shaded-hadoop-2</artifactId>
-				<version>${hadoop.version}-${flink.shaded.version}</version>
+				<groupId>org.apache.hadoop</groupId>
+				<artifactId>hadoop-common</artifactId>
+				<version>${hadoop.version}</version>
 				<exclusions>
 					<exclusion>
 						<groupId>log4j</groupId>
@@ -324,9 +324,57 @@ under the License.
 			</dependency>
 
 			<dependency>
-				<groupId>org.apache.flink</groupId>
-				<artifactId>flink-shaded-hadoop-2-uber</artifactId>
-				<version>${hadoop.version}-${flink.shaded.version}</version>
+				<groupId>org.apache.hadoop</groupId>
+				<artifactId>hadoop-hdfs</artifactId>
+				<version>${hadoop.version}</version>
+				<exclusions>
+					<exclusion>
+						<groupId>log4j</groupId>
+						<artifactId>log4j</artifactId>
+					</exclusion>
+					<exclusion>
+						<groupId>org.slf4j</groupId>
+						<artifactId>slf4j-log4j12</artifactId>
+					</exclusion>
+				</exclusions>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.hadoop</groupId>
+				<artifactId>hadoop-mapreduce-client-core</artifactId>
+				<version>${hadoop.version}</version>
+				<exclusions>
+					<exclusion>
+						<groupId>log4j</groupId>
+						<artifactId>log4j</artifactId>
+					</exclusion>
+					<exclusion>
+						<groupId>org.slf4j</groupId>
+						<artifactId>slf4j-log4j12</artifactId>
+					</exclusion>
+				</exclusions>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.hadoop</groupId>
+				<artifactId>hadoop-yarn-common</artifactId>
+				<version>${hadoop.version}</version>
+				<exclusions>
+					<exclusion>
+						<groupId>log4j</groupId>
+						<artifactId>log4j</artifactId>
+					</exclusion>
+					<exclusion>
+						<groupId>org.slf4j</groupId>
+						<artifactId>slf4j-log4j12</artifactId>
+					</exclusion>
+				</exclusions>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.hadoop</groupId>
+				<artifactId>hadoop-yarn-client</artifactId>
+				<version>${hadoop.version}</version>
 				<exclusions>
 					<exclusion>
 						<groupId>log4j</groupId>
@@ -1482,6 +1530,7 @@ under the License.
 						<forkNumber>0${surefire.forkNumber}</forkNumber>
 						<log4j.configuration>${log4j.configuration}</log4j.configuration>
 						<jobmanager.scheduler>${test.scheduler.type}</jobmanager.scheduler>
+						<hadoop.version>${hadoop.version}</hadoop.version>
 					</systemPropertyVariables>
 					<argLine>-Xms256m -Xmx2048m -Dmvn.forkNumber=${surefire.forkNumber} -XX:+UseG1GC</argLine>
 				</configuration>


[flink] 08/08: [FLINK-11086] Clean up profiles and dependency exclusions

Posted by rm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rmetzger pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 47323a4c898b4226bc006f5e6c722a36c89aa31d
Author: Robert Metzger <rm...@apache.org>
AuthorDate: Mon May 11 11:46:50 2020 +0200

    [FLINK-11086] Clean up profiles and dependency exclusions
    
    This closes #11983
---
 azure-pipelines.yml                                |  2 +-
 flink-connectors/flink-hbase/pom.xml               | 45 ++++---------------
 .../hbase/util/HBaseTestingClusterAutoStarter.java |  2 +-
 .../flink-bucketing-sink-test/pom.xml              | 32 +++++++++++---
 .../tests/util/kafka/SQLClientKafkaITCase.java     | 12 ++---
 .../flink-end-to-end-tests-common/pom.xml          | 51 ++++++++++++++++++++++
 .../apache/flink/tests/util/categories/Hadoop.java | 25 -----------
 flink-end-to-end-tests/pom.xml                     | 24 ----------
 flink-end-to-end-tests/run-nightly-tests.sh        |  8 ++--
 flink-formats/flink-orc-nohive/pom.xml             | 22 ----------
 flink-formats/flink-orc/pom.xml                    |  7 ---
 flink-tests/pom.xml                                | 10 +++--
 .../yarn/YARNSessionCapacitySchedulerITCase.java   |  3 +-
 .../java/org/apache/flink/yarn/YarnTestBase.java   |  1 -
 tools/azure-pipelines/build-apache-repo.yml        | 23 +++-------
 15 files changed, 112 insertions(+), 155 deletions(-)

diff --git a/azure-pipelines.yml b/azure-pipelines.yml
index d1966a9..ac76cfe 100644
--- a/azure-pipelines.yml
+++ b/azure-pipelines.yml
@@ -65,7 +65,7 @@ jobs:
         vmImage: 'ubuntu-latest'
       e2e_pool_definition:
         vmImage: 'ubuntu-16.04'
-      environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11 -Pe2e-hadoop"
+      environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
       run_end_to_end: false
       container: flink-build-container
       jdk: jdk8
diff --git a/flink-connectors/flink-hbase/pom.xml b/flink-connectors/flink-hbase/pom.xml
index cd31cfb..46b7ff9 100644
--- a/flink-connectors/flink-hbase/pom.xml
+++ b/flink-connectors/flink-hbase/pom.xml
@@ -101,15 +101,15 @@ under the License.
 		</dependency>
 
 		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-common</artifactId>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
+			<version>${project.version}</version>
 			<scope>provided</scope>
 		</dependency>
 
 		<dependency>
-			<groupId>org.apache.flink</groupId>
-			<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
-			<version>${project.version}</version>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
 			<scope>provided</scope>
 		</dependency>
 
@@ -145,31 +145,6 @@ under the License.
 					<groupId>org.mortbay.jetty</groupId>
 					<artifactId>servlet-api-2.5</artifactId>
 				</exclusion>
-				<!-- The hadoop dependencies are handled through flink-shaded-hadoop -->
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-common</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-auth</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-annotations</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-mapreduce-client-core</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-client</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-hdfs</artifactId>
-				</exclusion>
 				<!-- Bug in hbase annotations, can be removed when fixed. See FLINK-2153. -->
 				<exclusion>
 					<groupId>org.apache.hbase</groupId>
@@ -207,6 +182,10 @@ under the License.
 					<groupId>log4j</groupId>
 					<artifactId>log4j</artifactId>
 				</exclusion>
+				<exclusion>
+					<groupId>org.slf4j</groupId>
+					<artifactId>slf4j-log4j12</artifactId>
+				</exclusion>
 			</exclusions>
 		</dependency>
 
@@ -231,12 +210,6 @@ under the License.
 			<artifactId>flink-clients_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>test</scope>
-			<exclusions>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-core</artifactId>
-				</exclusion>
-			</exclusions>
 		</dependency>
 
 		<dependency>
diff --git a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/util/HBaseTestingClusterAutoStarter.java b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/util/HBaseTestingClusterAutoStarter.java
index 0fe8ad4..73648fa 100644
--- a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/util/HBaseTestingClusterAutoStarter.java
+++ b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/util/HBaseTestingClusterAutoStarter.java
@@ -146,7 +146,7 @@ public abstract class HBaseTestingClusterAutoStarter extends AbstractTestBase {
 	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.
+		// There is no Guava version in between that works with both.
 		Assume.assumeTrue("This test is skipped for Hadoop versions above 3", VersionUtil.compareVersions(System.getProperty("hadoop.version"), "3.0.0") < 0);
 
 		LOG.info("HBase minicluster: Starting");
diff --git a/flink-end-to-end-tests/flink-bucketing-sink-test/pom.xml b/flink-end-to-end-tests/flink-bucketing-sink-test/pom.xml
index 433215c..414ca3d 100644
--- a/flink-end-to-end-tests/flink-bucketing-sink-test/pom.xml
+++ b/flink-end-to-end-tests/flink-bucketing-sink-test/pom.xml
@@ -50,19 +50,39 @@
 			<groupId>org.apache.hadoop</groupId>
 			<artifactId>hadoop-common</artifactId>
 			<scope>provided</scope>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-hdfs</artifactId>
-			<scope>provided</scope>
 			<exclusions>
-				<!-- Needed for proper dependency convergence -->
+				<!-- needed for dependency convergence -->
 				<exclusion>
 					<groupId>commons-beanutils</groupId>
 					<artifactId>commons-beanutils</artifactId>
 				</exclusion>
+				<exclusion>
+					<groupId>org.codehaus.jackson</groupId>
+					<artifactId>jackson-core-asl</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.codehaus.jackson</groupId>
+					<artifactId>jackson-mapper-asl</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.google.guava</groupId>
+					<artifactId>guava</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>javax.servlet</groupId>
+					<artifactId>servlet-api</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.nimbusds</groupId>
+					<artifactId>nimbus-jose-jwt</artifactId>
+				</exclusion>
 			</exclusions>
 		</dependency>
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-hdfs</artifactId>
+			<scope>provided</scope>
+		</dependency>
 	</dependencies>
 
 	<build>
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java
index df3dec3..bf7358d 100644
--- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java
@@ -35,6 +35,7 @@ import org.apache.flink.shaded.guava18.com.google.common.base.Charsets;
 import org.apache.commons.lang3.StringUtils;
 import org.junit.Assert;
 import org.junit.Before;
+import org.junit.ClassRule;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -95,7 +96,8 @@ public class SQLClientKafkaITCase extends TestLogger {
 	private Path result;
 	private Path sqlClientSessionConf;
 
-	private static final DownloadCache downloadCache = DownloadCache.get();
+	@ClassRule
+	public static final DownloadCache DOWNLOAD_CACHE = DownloadCache.get();
 
 	private static final Path sqlAvroJar = TestUtils.getResourceJar(".*avro.jar");
 	private static final Path sqlJsonJar = TestUtils.getResourceJar(".*json.jar");
@@ -112,15 +114,15 @@ public class SQLClientKafkaITCase extends TestLogger {
 
 	@Before
 	public void before() throws Exception {
-		downloadCache.before();
+		DOWNLOAD_CACHE.before();
 		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));
-		apacheAvroJars.add(downloadCache.getOrDownload("https://repo1.maven.org/maven2/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar", tmpPath));
-		apacheAvroJars.add(downloadCache.getOrDownload("https://repo1.maven.org/maven2/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar", tmpPath));
+		apacheAvroJars.add(DOWNLOAD_CACHE.getOrDownload("https://repo1.maven.org/maven2/org/apache/avro/avro/1.8.2/avro-1.8.2.jar", tmpPath));
+		apacheAvroJars.add(DOWNLOAD_CACHE.getOrDownload("https://repo1.maven.org/maven2/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar", tmpPath));
+		apacheAvroJars.add(DOWNLOAD_CACHE.getOrDownload("https://repo1.maven.org/maven2/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar", tmpPath));
 	}
 
 	@Test
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml
index 25092e5..410dbbe 100644
--- a/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml
@@ -67,6 +67,57 @@ under the License.
 			<artifactId>flink-dist_${scala.binary.version}</artifactId>
 			<version>${project.version}</version>
 			<scope>provided</scope>
+			<exclusions>
+				<!-- needed for dependency convergence -->
+				<exclusion>
+					<groupId>commons-beanutils</groupId>
+					<artifactId>commons-beanutils</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.codehaus.jackson</groupId>
+					<artifactId>jackson-core-asl</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.codehaus.jackson</groupId>
+					<artifactId>jackson-mapper-asl</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.codehaus.jackson</groupId>
+					<artifactId>jackson-xc</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>org.codehaus.jackson</groupId>
+					<artifactId>jackson-jaxrs</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.google.guava</groupId>
+					<artifactId>guava</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.squareup.okio</groupId>
+					<artifactId>okio</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>javax.servlet</groupId>
+					<artifactId>servlet-api</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.nimbusds</groupId>
+					<artifactId>nimbus-jose-jwt</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.nimbusds</groupId>
+					<artifactId>nimbus-jose-jwt</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.google.inject</groupId>
+					<artifactId>guice</artifactId>
+				</exclusion>
+				<exclusion>
+					<groupId>com.google.inject.extensions</groupId>
+					<artifactId>guice-servlet</artifactId>
+				</exclusion>
+			</exclusions>
 		</dependency>
 	</dependencies>
 
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/Hadoop.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/Hadoop.java
deleted file mode 100644
index b28ad87..0000000
--- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/categories/Hadoop.java
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.tests.util.categories;
-
-/**
- * Marker interface for tests that require Hadoop.
- */
-public interface Hadoop {
-}
diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml
index dbdcf49..87b5fb1 100644
--- a/flink-end-to-end-tests/pom.xml
+++ b/flink-end-to-end-tests/pom.xml
@@ -45,7 +45,6 @@ under the License.
 		<e2e.include.org.apache.flink.tests.util.categories.TravisGroup5>false</e2e.include.org.apache.flink.tests.util.categories.TravisGroup5>
 		<e2e.include.org.apache.flink.tests.util.categories.TravisGroup6>false</e2e.include.org.apache.flink.tests.util.categories.TravisGroup6>
 		<e2e.include.org.apache.flink.tests.util.categories.PreCommit>false</e2e.include.org.apache.flink.tests.util.categories.PreCommit>
-		<e2e.exclude.org.apache.flink.tests.util.categories.Hadoop>true</e2e.exclude.org.apache.flink.tests.util.categories.Hadoop>
 		<e2e.exclude.org.apache.flink.testutils.junit.FailsOnJava11>false</e2e.exclude.org.apache.flink.testutils.junit.FailsOnJava11>
 	</properties>
 
@@ -152,12 +151,6 @@ under the License.
 			</properties>
 		</profile>
 		<profile>
-			<id>e2e-hadoop</id>
-			<properties>
-				<e2e.exclude.org.apache.flink.tests.util.categories.Hadoop>false</e2e.exclude.org.apache.flink.tests.util.categories.Hadoop>
-			</properties>
-		</profile>
-		<profile>
 			<id>java11</id>
 			<activation>
 				<jdk>11</jdk>
@@ -172,11 +165,9 @@ under the License.
 		<plugins>
 			<plugin>
 				<artifactId>maven-resources-plugin</artifactId>
-				<!-- <version>3.1.0</version> -->
 				<executions>
 					<execution>
 						<id>copy-resources</id>
-						<!-- here the phase you need -->
 						<phase>package</phase>
 						<goals>
 							<goal>copy-resources</goal>
@@ -298,21 +289,6 @@ under the License.
 					</execution>
 				</executions>
 			</plugin>
-			<plugin>
-				<groupId>org.apache.maven.plugins</groupId>
-				<artifactId>maven-enforcer-plugin</artifactId>
-				<executions>
-					<execution>
-						<id>dependency-convergence</id>
-						<goals>
-							<goal>enforce</goal>
-						</goals>
-						<configuration>
-							<skip>true</skip>
-						</configuration>
-					</execution>
-				</executions>
-			</plugin>
 		</plugins>
 
 		<pluginManagement>
diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh
index e8c3d33..500c067 100755
--- a/flink-end-to-end-tests/run-nightly-tests.sh
+++ b/flink-end-to-end-tests/run-nightly-tests.sh
@@ -135,10 +135,8 @@ if [[ ${PROFILE} != *"jdk11"* ]]; then
     run_test "Running Kerberized YARN per-job on Docker test (custom fs plugin)" "$END_TO_END_DIR/test-scripts/test_yarn_job_kerberos_docker.sh dummy-fs"
     run_test "Running Kerberized YARN application on Docker test (default input)" "$END_TO_END_DIR/test-scripts/test_yarn_application_kerberos_docker.sh"
     run_test "Running Kerberized YARN application on Docker test (custom fs plugin)" "$END_TO_END_DIR/test-scripts/test_yarn_application_kerberos_docker.sh dummy-fs"
-    if [[ $PROFILE == *"include-hadoop"* ]]; then
-			run_test "Run Mesos WordCount test" "$END_TO_END_DIR/test-scripts/test_mesos_wordcount.sh"
-			run_test "Run Mesos multiple submission test" "$END_TO_END_DIR/test-scripts/test_mesos_multiple_submissions.sh"
-		fi
+	run_test "Run Mesos WordCount test" "$END_TO_END_DIR/test-scripts/test_mesos_wordcount.sh"
+	run_test "Run Mesos multiple submission test" "$END_TO_END_DIR/test-scripts/test_mesos_multiple_submissions.sh"
 	fi
 
 	run_test "Test PubSub connector with Docker based Google PubSub Emulator" "$END_TO_END_DIR/test-scripts/test_streaming_gcp_pubsub.sh"
@@ -174,7 +172,7 @@ run_test "Batch SQL end-to-end test" "$END_TO_END_DIR/test-scripts/test_batch_sq
 run_test "Streaming SQL end-to-end test (Old planner)" "$END_TO_END_DIR/test-scripts/test_streaming_sql.sh old" "skip_check_exceptions"
 run_test "Streaming SQL end-to-end test (Blink planner)" "$END_TO_END_DIR/test-scripts/test_streaming_sql.sh blink" "skip_check_exceptions"
 # skip test if hadoop version is 2.4.1 (FLINK-16629)
-if [[ $PROFILE == *"include-hadoop"* && $PROFILE != *"hadoop.version=2.4.1"* ]]; then
+if [[ $PROFILE != *"hadoop.version=2.4.1"* ]]; then
 	run_test "Streaming bucketing end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_bucketing.sh" "skip_check_exceptions"
 fi
 run_test "Streaming File Sink end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_file_sink.sh" "skip_check_exceptions"
diff --git a/flink-formats/flink-orc-nohive/pom.xml b/flink-formats/flink-orc-nohive/pom.xml
index f352313..90cae93 100644
--- a/flink-formats/flink-orc-nohive/pom.xml
+++ b/flink-formats/flink-orc-nohive/pom.xml
@@ -63,34 +63,12 @@ under the License.
 			<version>${orc.version}</version>
 			<classifier>nohive</classifier>
 			<exclusions>
-				<!-- Exclude ORC's Hadoop dependency and pull in Flink's Hadoop. -->
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-common</artifactId>
-				</exclusion>
-				<exclusion>
-					<groupId>org.apache.hadoop</groupId>
-					<artifactId>hadoop-hdfs</artifactId>
-				</exclusion>
 				<exclusion>
 					<groupId>org.apache.hive</groupId>
 					<artifactId>hive-storage-api</artifactId>
 				</exclusion>
 			</exclusions>
 		</dependency>
-
-		<!-- Replacement for ORC's Hadoop dependency. -->
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-common</artifactId>
-			<scope>provided</scope>
-		</dependency>
-
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-hdfs</artifactId>
-			<scope>provided</scope>
-		</dependency>
 		
 		<!-- test dependencies -->
 		<dependency>
diff --git a/flink-formats/flink-orc/pom.xml b/flink-formats/flink-orc/pom.xml
index 00a39e2..510ac4e 100644
--- a/flink-formats/flink-orc/pom.xml
+++ b/flink-formats/flink-orc/pom.xml
@@ -86,13 +86,6 @@ under the License.
 			</exclusions>
 		</dependency>
 
-		<!-- Replacement for ORC's Hadoop dependency. -->
-		<dependency>
-			<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-common</artifactId>
-			<scope>provided</scope>
-		</dependency>
-
 		<dependency>
 			<groupId>org.apache.commons</groupId>
 			<artifactId>commons-lang3</artifactId>
diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml
index 53f781c..7e7e63b 100644
--- a/flink-tests/pom.xml
+++ b/flink-tests/pom.xml
@@ -58,15 +58,17 @@ under the License.
 			<scope>test</scope>
 		</dependency>
 
-		<groupId>org.apache.hadoop</groupId>
-			<artifactId>hadoop-common</artifactId>
-			<scope>test</scope>
-</dependency>
 		<dependency>
 			<groupId>org.apache.flink</groupId>
 			<artifactId>flink-shaded-jackson</artifactId>
 			<scope>test</scope>
 		</dependency>
+		
+		<dependency>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-common</artifactId>
+			<scope>test</scope>
+		</dependency>
 
 		<dependency>
 			<groupId>org.apache.flink</groupId>
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
index 490c858..fe1e62a 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
@@ -155,7 +155,8 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 	public void testStartYarnSessionClusterInQaTeamQueue() throws Exception {
 		runTest(() -> runWithArgs(new String[]{
 				"-j", flinkUberjar.getAbsolutePath(),
-				"-t", flinkLibFolder.getAbsolutePath(), "-jm", "768m",
+				"-t", flinkLibFolder.getAbsolutePath(),
+				"-jm", "768m",
 				"-tm", "1024m", "-qu", "qa-team"},
 			"JobManager Web Interface:", null, RunTypes.YARN_SESSION, 0));
 	}
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
index 003c7f7..21380d7 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
@@ -338,7 +338,6 @@ public abstract class YarnTestBase extends TestLogger {
 				true);
 		yarnClusterDescriptor.setLocalJarPath(new Path(flinkUberjar.toURI()));
 		yarnClusterDescriptor.addShipFiles(Collections.singletonList(flinkLibFolder));
-		yarnClusterDescriptor.addShipFiles(Collections.singletonList(flinkShadedHadoopDir));
 		return yarnClusterDescriptor;
 	}
 
diff --git a/tools/azure-pipelines/build-apache-repo.yml b/tools/azure-pipelines/build-apache-repo.yml
index b00fc30..dab6d95 100644
--- a/tools/azure-pipelines/build-apache-repo.yml
+++ b/tools/azure-pipelines/build-apache-repo.yml
@@ -63,7 +63,7 @@ stages:
             name: Default
           e2e_pool_definition:
             vmImage: 'ubuntu-16.04'
-          environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11 -Pe2e-hadoop"
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
           run_end_to_end: false
           container: flink-build-container
           jdk: jdk8
@@ -80,7 +80,7 @@ stages:
             name: Default
           e2e_pool_definition:
             vmImage: 'ubuntu-16.04'
-          environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.4.1 -Pskip-hive-tests -Pe2e-hadoop"
+          environment: PROFILE="-Dhadoop.version=2.4.1 -Pskip-hive-tests"
           run_end_to_end: true
           container: flink-build-container
           jdk: jdk8
@@ -91,7 +91,7 @@ stages:
             name: Default
           e2e_pool_definition:
             vmImage: 'ubuntu-16.04'
-          environment: PROFILE="-Dinclude-hadoop -Dinclude_hadoop_aws -Dhadoop.version=3.1.3 -Pe2e-hadoop -Phadoop3-tests"
+          environment: PROFILE="-Dinclude_hadoop_aws -Dhadoop.version=3.1.3 -Phadoop3-tests"
           run_end_to_end: true
           container: flink-build-container
           jdk: jdk8
@@ -102,7 +102,7 @@ stages:
             name: Default
           e2e_pool_definition:
             vmImage: 'ubuntu-16.04'
-          environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.12 -Phive-1.2.1 -Pe2e-hadoop"
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.12 -Phive-1.2.1"
           run_end_to_end: true
           container: flink-build-container
           jdk: jdk8
@@ -113,21 +113,10 @@ stages:
             name: Default
           e2e_pool_definition:
             vmImage: 'ubuntu-16.04'
-          environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11 -Djdk11 -Pe2e-hadoop"
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11 -Djdk11"
           run_end_to_end: true
           container: flink-build-container
           jdk: jdk11
-      - template: jobs-template.yml
-        parameters:
-          stage_name: cron_hadoopfree
-          test_pool_definition:
-            name: Default
-          e2e_pool_definition:
-            vmImage: 'ubuntu-16.04'
-          environment: PROFILE=""
-          run_end_to_end: true
-          container: flink-build-container
-          jdk: jdk8
       - job: docs_404_check # run on a MSFT provided machine
         pool: 
           vmImage: 'ubuntu-16.04'
@@ -140,5 +129,5 @@ stages:
       - template: build-python-wheels.yml
         parameters:
           stage_name: cron_python_wheels
-          environment: PROFILE="-Dinclude-hadoop -Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
           container: flink-build-container


[flink] 02/08: [FLINK-11086][e2e] Properly add Avro JARs in SQLClientKafkaITCase

Posted by rm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rmetzger pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 07330dd6aeabf84d450440614dcb0117bfd1c9c7
Author: Robert Metzger <rm...@apache.org>
AuthorDate: Mon May 4 10:11:58 2020 +0200

    [FLINK-11086][e2e] Properly add Avro JARs in SQLClientKafkaITCase
---
 .../flink/tests/util/kafka/SQLClientKafkaITCase.java   | 18 +++++++++++++++---
 .../flink/tests/util/flink/SQLJobSubmission.java       |  5 +++++
 2 files changed, 20 insertions(+), 3 deletions(-)

diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java
index 923377e..df3dec3 100644
--- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/src/test/java/org/apache/flink/tests/util/kafka/SQLClientKafkaITCase.java
@@ -19,7 +19,7 @@
 package org.apache.flink.tests.util.kafka;
 
 import org.apache.flink.tests.util.TestUtils;
-import org.apache.flink.tests.util.categories.Hadoop;
+import org.apache.flink.tests.util.cache.DownloadCache;
 import org.apache.flink.tests.util.categories.TravisGroup1;
 import org.apache.flink.tests.util.flink.ClusterController;
 import org.apache.flink.tests.util.flink.FlinkResource;
@@ -51,9 +51,11 @@ import java.net.URL;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.StandardOpenOption;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import static org.hamcrest.Matchers.arrayContainingInAnyOrder;
@@ -63,7 +65,7 @@ import static org.junit.Assert.assertThat;
  * End-to-end test for the kafka SQL connectors.
  */
 @RunWith(Parameterized.class)
-@Category(value = {TravisGroup1.class, FailsOnJava11.class, Hadoop.class})
+@Category(value = {TravisGroup1.class, FailsOnJava11.class})
 public class SQLClientKafkaITCase extends TestLogger {
 
 	private static final Logger LOG = LoggerFactory.getLogger(SQLClientKafkaITCase.class);
@@ -93,9 +95,12 @@ public class SQLClientKafkaITCase extends TestLogger {
 	private Path result;
 	private Path sqlClientSessionConf;
 
+	private static final DownloadCache downloadCache = DownloadCache.get();
+
 	private static final Path sqlAvroJar = TestUtils.getResourceJar(".*avro.jar");
 	private static final Path sqlJsonJar = TestUtils.getResourceJar(".*json.jar");
 	private static final Path sqlToolBoxJar = TestUtils.getResourceJar(".*SqlToolbox.jar");
+	private final List<Path> apacheAvroJars = new ArrayList<>();
 	private final Path sqlConnectorKafkaJar;
 
 	public SQLClientKafkaITCase(String kafkaVersion, String kafkaSQLVersion, String kafkaSQLJarPattern) {
@@ -106,11 +111,16 @@ public class SQLClientKafkaITCase extends TestLogger {
 	}
 
 	@Before
-	public void before() {
+	public void before() throws Exception {
+		downloadCache.before();
 		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));
+		apacheAvroJars.add(downloadCache.getOrDownload("https://repo1.maven.org/maven2/org/codehaus/jackson/jackson-core-asl/1.9.13/jackson-core-asl-1.9.13.jar", tmpPath));
+		apacheAvroJars.add(downloadCache.getOrDownload("https://repo1.maven.org/maven2/org/codehaus/jackson/jackson-mapper-asl/1.9.13/jackson-mapper-asl-1.9.13.jar", tmpPath));
 	}
 
 	@Test
@@ -179,6 +189,7 @@ public class SQLClientKafkaITCase extends TestLogger {
 
 		clusterController.submitSQLJob(new SQLJobSubmission.SQLJobSubmissionBuilder(sqlStatement1)
 				.addJar(sqlAvroJar)
+				.addJars(apacheAvroJars)
 				.addJar(sqlJsonJar)
 				.addJar(sqlConnectorKafkaJar)
 				.addJar(sqlToolBoxJar)
@@ -194,6 +205,7 @@ public class SQLClientKafkaITCase extends TestLogger {
 
 		clusterController.submitSQLJob(new SQLJobSubmission.SQLJobSubmissionBuilder(sqlStatement2)
 				.addJar(sqlAvroJar)
+				.addJars(apacheAvroJars)
 				.addJar(sqlJsonJar)
 				.addJar(sqlConnectorKafkaJar)
 				.addJar(sqlToolBoxJar)
diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/SQLJobSubmission.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/SQLJobSubmission.java
index cfb3e39..1f4013a 100644
--- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/SQLJobSubmission.java
+++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/SQLJobSubmission.java
@@ -90,6 +90,11 @@ public class SQLJobSubmission {
 			return this;
 		}
 
+		public SQLJobSubmissionBuilder addJars(List<Path> jarFiles) {
+			jarFiles.forEach(this::addJar);
+			return this;
+		}
+
 		public SQLJobSubmission build() {
 			return new SQLJobSubmission(sql, jars, defaultEnvFile, sessionEnvFile);
 		}


[flink] 07/08: [FLINK-11086][docs] Make HADOOP_CLASSPATH approach more prominent in docs

Posted by rm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rmetzger pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 2cc63a6d0f6cec23df5b00797a3876907ecb3342
Author: Robert Metzger <rm...@apache.org>
AuthorDate: Mon May 4 12:09:48 2020 +0200

    [FLINK-11086][docs] Make HADOOP_CLASSPATH approach more prominent in docs
---
 docs/ops/deployment/hadoop.md        | 81 +++++++++++++++++-------------------
 docs/ops/deployment/hadoop.zh.md     | 81 +++++++++++++++++-------------------
 docs/ops/deployment/yarn_setup.md    | 23 ++++------
 docs/ops/deployment/yarn_setup.zh.md | 27 +++++-------
 4 files changed, 96 insertions(+), 116 deletions(-)

diff --git a/docs/ops/deployment/hadoop.md b/docs/ops/deployment/hadoop.md
index 24471853..71914a6 100644
--- a/docs/ops/deployment/hadoop.md
+++ b/docs/ops/deployment/hadoop.md
@@ -26,34 +26,13 @@ under the License.
 * This will be replaced by the TOC
 {:toc}
 
-## Referencing a Hadoop configuration
-
-You can reference a Hadoop configuration by setting the environment variable `HADOOP_CONF_DIR`.
-
-```sh
-HADOOP_CONF_DIR=/path/to/etc/hadoop
-```
-
-Referencing the HDFS configuration in the [Flink configuration]({{ site.baseurl }}/ops/config.html#hdfs) is deprecated.
-
-Another way to provide the Hadoop configuration is to have it on the class path of the Flink process, see more details below.
 
 ## Providing Hadoop classes
 
 In order to use Hadoop features (e.g., YARN, HDFS) it is necessary to provide Flink with the required Hadoop classes,
 as these are not bundled by default.
 
-This can be done by 
-1) Adding the Hadoop classpath to Flink
-2) Putting the required jar files into /lib directory of the Flink distribution
-Option 1) requires very little work, integrates nicely with existing Hadoop setups and should be the
-preferred approach.
-However, Hadoop has a large dependency footprint that increases the risk for dependency conflicts to occur.
-If this happens, please refer to option 2).
-
-The following subsections explains these approaches in detail.
-
-### Adding Hadoop Classpaths
+This can be done by adding the Hadoop classpath to Flink through the `HADOOP_CLASSPATH` environment variable.
 
 Flink will use the environment variable `HADOOP_CLASSPATH` to augment the
 classpath that is used when starting Flink components such as the Client,
@@ -75,35 +54,24 @@ in the shell. Note that `hadoop` is the hadoop binary and that `classpath` is an
 
 Putting the Hadoop configuration in the same class path as the Hadoop libraries makes Flink pick up that configuration.
 
-### Adding Hadoop to /lib
-
-The Flink project releases Hadoop distributions for specific versions, that relocate or exclude several dependencies
-to reduce the risk of dependency clashes.
-These can be found in the [Additional Components]({{ site.download_url }}#additional-components) section of the download page.
-For these versions it is sufficient to download the corresponding `Pre-bundled Hadoop` component and putting it into
-the `/lib` directory of the Flink distribution.
-
-If the used Hadoop version is not listed on the download page (possibly due to being a Vendor-specific version),
-then it is necessary to build [flink-shaded](https://github.com/apache/flink-shaded) against this version.
-You can find the source code for this project in the [Additional Components]({{ site.download_url }}#additional-components) section of the download page.
+## Referencing a Hadoop configuration
 
-<span class="label label-info">Note</span> If you want to build `flink-shaded` against a vendor specific Hadoop version, you first have to configure the
-vendor-specific maven repository in your local maven setup as described [here](https://maven.apache.org/guides/mini/guide-multiple-repositories.html).
+You can reference a Hadoop configuration by setting the environment variable `HADOOP_CONF_DIR`.
 
-Run the following command to build and install `flink-shaded` against your desired Hadoop version (e.g., for version `2.6.5-custom`):
+```sh
+HADOOP_CONF_DIR=/path/to/etc/hadoop
+```
 
-{% highlight bash %}
-mvn clean install -Dhadoop.version=2.6.5-custom
-{% endhighlight %}
+Referencing the HDFS configuration in the [Flink configuration]({{ site.baseurl }}/ops/config.html#hdfs) is deprecated.
 
-After this step is complete, put the `flink-shaded-hadoop-2-uber` jar into the `/lib` directory of the Flink distribution.
+Another way to provide the Hadoop configuration is to have it on the class path of the Flink process, see more details above.
 
 ## Running a job locally
 
 To run a job locally as one JVM process using the mini cluster, the required hadoop dependencies have to be explicitly
 added to the classpath of the started JVM process.
 
-To run an application using maven (also from IDE as a maven project), the required hadoop dependencies can be added
+To run an application using Maven (also from IDE as a Maven project), the required Hadoop dependencies can be added
 as provided to the pom.xml, e.g.:
 
 ```xml
@@ -115,9 +83,38 @@ as provided to the pom.xml, e.g.:
 </dependency>
 ```
 
-This way it should work both in local and cluster run where the provided dependencies are added elsewhere as described before.
+This way it should work both in local and cluster mode where the provided dependencies are added elsewhere as described before.
 
 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.
 
+
+## Using `flink-shaded-hadoop-2-uber` jar for resolving dependency conflicts (legacy)
+
+<div class="alert alert-info" markdown="span">
+  <strong>Warning:</strong> Starting from Flink 1.11, using `flink-shaded-hadoop-2-uber` releases is not officially supported
+  by the Flink project anymore. Users are advised to provide Hadoop dependencies through `HADOOP_CLASSPATH` (see above).
+</div>
+
+The Flink project used to (until Flink 1.10) release Hadoop distributions for specific versions, that relocate or exclude several dependencies to reduce the risk of dependency clashes.
+These can be found in the [Additional Components]({{ site.download_url }}#additional-components) section of the download page.
+For these versions it is sufficient to download the corresponding `Pre-bundled Hadoop` component and putting it into
+the `/lib` directory of the Flink distribution.
+
+If the used Hadoop version is not listed on the download page (possibly due to being a Vendor-specific version),
+then it is necessary to build [flink-shaded](https://github.com/apache/flink-shaded) against this version.
+You can find the source code for this project in the [Additional Components]({{ site.download_url }}#additional-components) section of the download page.
+
+<span class="label label-info">Note</span> If you want to build `flink-shaded` against a vendor specific Hadoop version, you first have to configure the
+vendor-specific maven repository in your local maven setup as described [here](https://maven.apache.org/guides/mini/guide-multiple-repositories.html).
+
+Run the following command to build and install `flink-shaded` against your desired Hadoop version (e.g., for version `2.6.5-custom`):
+
+{% highlight bash %}
+mvn clean install -Dhadoop.version=2.6.5-custom
+{% endhighlight %}
+
+After this step is complete, put the `flink-shaded-hadoop-2-uber` jar into the `/lib` directory of the Flink distribution.
+
+
 {% top %}
diff --git a/docs/ops/deployment/hadoop.zh.md b/docs/ops/deployment/hadoop.zh.md
index 39c54cc..f9cf3e1 100644
--- a/docs/ops/deployment/hadoop.zh.md
+++ b/docs/ops/deployment/hadoop.zh.md
@@ -26,34 +26,13 @@ under the License.
 * This will be replaced by the TOC
 {:toc}
 
-## Referencing a Hadoop configuration
-
-You can reference a Hadoop configuration by setting the environment variable `HADOOP_CONF_DIR`.
-
-```sh
-HADOOP_CONF_DIR=/path/to/etc/hadoop
-```
-
-Referencing the HDFS configuration in the [Flink configuration]({{ site.baseurl }}/ops/config.html#hdfs) is deprecated.
-
-Another way to provide the Hadoop configuration is to have it on the class path of the Flink process, see more details below.
 
 ## Providing Hadoop classes
 
 In order to use Hadoop features (e.g., YARN, HDFS) it is necessary to provide Flink with the required Hadoop classes,
 as these are not bundled by default.
 
-This can be done by 
-1) Adding the Hadoop classpath to Flink
-2) Putting the required jar files into /lib directory of the Flink distribution
-Option 1) requires very little work, integrates nicely with existing Hadoop setups and should be the
-preferred approach.
-However, Hadoop has a large dependency footprint that increases the risk for dependency conflicts to occur.
-If this happens, please refer to option 2).
-
-The following subsections explains these approaches in detail.
-
-### Adding Hadoop Classpaths
+This can be done by adding the Hadoop classpath to Flink through the `HADOOP_CLASSPATH` environment variable.
 
 Flink will use the environment variable `HADOOP_CLASSPATH` to augment the
 classpath that is used when starting Flink components such as the Client,
@@ -75,35 +54,24 @@ in the shell. Note that `hadoop` is the hadoop binary and that `classpath` is an
 
 Putting the Hadoop configuration in the same class path as the Hadoop libraries makes Flink pick up that configuration.
 
-### Adding Hadoop to /lib
-
-The Flink project releases Hadoop distributions for specific versions, that relocate or exclude several dependencies
-to reduce the risk of dependency clashes.
-These can be found in the [Additional Components]({{ site.download_url }}#additional-components) section of the download page.
-For these versions it is sufficient to download the corresponding `Pre-bundled Hadoop` component and putting it into
-the `/lib` directory of the Flink distribution.
-
-If the used Hadoop version is not listed on the download page (possibly due to being a Vendor-specific version),
-then it is necessary to build [flink-shaded](https://github.com/apache/flink-shaded) against this version.
-You can find the source code for this project in the [Additional Components]({{ site.download_url }}#additional-components) section of the download page.
+## Referencing a Hadoop configuration
 
-<span class="label label-info">Note</span> If you want to build `flink-shaded` against a vendor specific Hadoop version, you first have to configure the
-vendor-specific maven repository in your local maven setup as described [here](https://maven.apache.org/guides/mini/guide-multiple-repositories.html).
+You can reference a Hadoop configuration by setting the environment variable `HADOOP_CONF_DIR`.
 
-Run the following command to build and install `flink-shaded` against your desired Hadoop version (e.g., for version `2.6.5-custom`):
+```sh
+HADOOP_CONF_DIR=/path/to/etc/hadoop
+```
 
-{% highlight bash %}
-mvn clean install -Dhadoop.version=2.6.5-custom
-{% endhighlight %}
+Referencing the HDFS configuration in the [Flink configuration]({{ site.baseurl }}/ops/config.html#hdfs) is deprecated.
 
-After this step is complete, put the `flink-shaded-hadoop-2-uber` jar into the `/lib` directory of the Flink distribution.
+Another way to provide the Hadoop configuration is to have it on the class path of the Flink process, see more details above.
 
 ## Running a job locally
 
 To run a job locally as one JVM process using the mini cluster, the required hadoop dependencies have to be explicitly
 added to the classpath of the started JVM process.
 
-To run an application using maven (also from IDE as a maven project), the required hadoop dependencies can be added
+To run an application using Maven (also from IDE as a Maven project), the required Hadoop dependencies can be added
 as provided to the pom.xml, e.g.:
 
 ```xml
@@ -115,9 +83,38 @@ as provided to the pom.xml, e.g.:
 </dependency>
 ```
 
-This way it should work both in local and cluster run where the provided dependencies are added elsewhere as described before.
+This way it should work both in local and cluster mode where the provided dependencies are added elsewhere as described before.
 
 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.
 
+
+## Using `flink-shaded-hadoop-2-uber` jar for resolving dependency conflicts (legacy)
+
+<div class="alert alert-info" markdown="span">
+  <strong>Warning:</strong> Starting from Flink 1.11, using `flink-shaded-hadoop-2-uber` releases is not officially supported
+  by the Flink project anymore. Users are advised to provide Hadoop dependencies through `HADOOP_CLASSPATH` (see above).
+</div>
+
+The Flink project used to (until Flink 1.10) release Hadoop distributions for specific versions, that relocate or exclude several dependencies to reduce the risk of dependency clashes.
+These can be found in the [Additional Components]({{ site.download_url }}#additional-components) section of the download page.
+For these versions it is sufficient to download the corresponding `Pre-bundled Hadoop` component and putting it into
+the `/lib` directory of the Flink distribution.
+
+If the used Hadoop version is not listed on the download page (possibly due to being a Vendor-specific version),
+then it is necessary to build [flink-shaded](https://github.com/apache/flink-shaded) against this version.
+You can find the source code for this project in the [Additional Components]({{ site.download_url }}#additional-components) section of the download page.
+
+<span class="label label-info">Note</span> If you want to build `flink-shaded` against a vendor specific Hadoop version, you first have to configure the
+vendor-specific maven repository in your local maven setup as described [here](https://maven.apache.org/guides/mini/guide-multiple-repositories.html).
+
+Run the following command to build and install `flink-shaded` against your desired Hadoop version (e.g., for version `2.6.5-custom`):
+
+{% highlight bash %}
+mvn clean install -Dhadoop.version=2.6.5-custom
+{% endhighlight %}
+
+After this step is complete, put the `flink-shaded-hadoop-2-uber` jar into the `/lib` directory of the Flink distribution.
+
+
 {% top %}
diff --git a/docs/ops/deployment/yarn_setup.md b/docs/ops/deployment/yarn_setup.md
index 2e04d45..4dcd290 100644
--- a/docs/ops/deployment/yarn_setup.md
+++ b/docs/ops/deployment/yarn_setup.md
@@ -33,11 +33,8 @@ under the License.
 Start a YARN session where the job manager gets 1 GB of heap space and the task managers 4 GB of heap space assigned:
 
 {% highlight bash %}
-# get the hadoop2 package from the Flink download page at
-# {{ site.download_url }}
-curl -O <flink_hadoop2_download_url>
-tar xvzf flink-{{ site.version }}-bin-hadoop2.tgz
-cd flink-{{ site.version }}/
+# If HADOOP_CLASSPATH is not set:
+#   export HADOOP_CLASSPATH=`hadoop classpath`
 ./bin/yarn-session.sh -jm 1024m -tm 4096m
 {% endhighlight %}
 
@@ -48,11 +45,8 @@ Once the session has been started, you can submit jobs to the cluster using the
 ### Run a Flink job on YARN
 
 {% highlight bash %}
-# get the hadoop2 package from the Flink download page at
-# {{ site.download_url }}
-curl -O <flink_hadoop2_download_url>
-tar xvzf flink-{{ site.version }}-bin-hadoop2.tgz
-cd flink-{{ site.version }}/
+# If HADOOP_CLASSPATH is not set:
+#   export HADOOP_CLASSPATH=`hadoop classpath`
 ./bin/flink run -m yarn-cluster -p 4 -yjm 1024m -ytm 4096m ./examples/batch/WordCount.jar
 {% endhighlight %}
 
@@ -62,11 +56,9 @@ Apache [Hadoop YARN](http://hadoop.apache.org/) is a cluster resource management
 
 **Requirements**
 
-- at least Apache Hadoop 2.2
+- at least Apache Hadoop 2.4.1
 - HDFS (Hadoop Distributed File System) (or another distributed file system supported by Hadoop)
 
-If you have troubles using the Flink YARN client, have a look in the [FAQ section](https://flink.apache.org/faq.html#yarn-deployment).
-
 ### Start Flink Session
 
 Follow these instructions to learn how to launch a Flink Session within your YARN cluster.
@@ -75,15 +67,16 @@ A session will start all required Flink services (JobManager and TaskManagers) s
 
 #### Download Flink
 
-Download a Flink package for Hadoop >= 2 from the [download page]({{ site.download_url }}). It contains the required files.
+Download a Flink package from the [download page]({{ site.download_url }}). It contains the required files.
 
 Extract the package using:
 
 {% highlight bash %}
-tar xvzf flink-{{ site.version }}-bin-hadoop2.tgz
+tar xvzf flink-{{ site.version }}-bin-scala*.tgz
 cd flink-{{site.version }}/
 {% endhighlight %}
 
+
 #### Start a Session
 
 Use the following command to start a session
diff --git a/docs/ops/deployment/yarn_setup.zh.md b/docs/ops/deployment/yarn_setup.zh.md
index 887bd71..7ff496b 100644
--- a/docs/ops/deployment/yarn_setup.zh.md
+++ b/docs/ops/deployment/yarn_setup.zh.md
@@ -33,11 +33,8 @@ under the License.
 Start a YARN session where the job manager gets 1 GB of heap space and the task managers 4 GB of heap space assigned:
 
 {% highlight bash %}
-# get the hadoop2 package from the Flink download page at
-# {{ site.download_url }}
-curl -O <flink_hadoop2_download_url>
-tar xvzf flink-{{ site.version }}-bin-hadoop2.tgz
-cd flink-{{ site.version }}/
+# If HADOOP_CLASSPATH is not set:
+#   export HADOOP_CLASSPATH=`hadoop classpath`
 ./bin/yarn-session.sh -jm 1024m -tm 4096m
 {% endhighlight %}
 
@@ -48,11 +45,8 @@ Once the session has been started, you can submit jobs to the cluster using the
 ### Run a Flink job on YARN
 
 {% highlight bash %}
-# get the hadoop2 package from the Flink download page at
-# {{ site.download_url }}
-curl -O <flink_hadoop2_download_url>
-tar xvzf flink-{{ site.version }}-bin-hadoop2.tgz
-cd flink-{{ site.version }}/
+# If HADOOP_CLASSPATH is not set:
+#   export HADOOP_CLASSPATH=`hadoop classpath`
 ./bin/flink run -m yarn-cluster -p 4 -yjm 1024m -ytm 4096m ./examples/batch/WordCount.jar
 {% endhighlight %}
 
@@ -62,11 +56,9 @@ Apache [Hadoop YARN](http://hadoop.apache.org/) is a cluster resource management
 
 **Requirements**
 
-- at least Apache Hadoop 2.2
+- at least Apache Hadoop 2.4.1
 - HDFS (Hadoop Distributed File System) (or another distributed file system supported by Hadoop)
 
-If you have troubles using the Flink YARN client, have a look in the [FAQ section](https://flink.apache.org/faq.html#yarn-deployment).
-
 ### Start Flink Session
 
 Follow these instructions to learn how to launch a Flink Session within your YARN cluster.
@@ -75,15 +67,16 @@ A session will start all required Flink services (JobManager and TaskManagers) s
 
 #### Download Flink
 
-Download a Flink package for Hadoop >= 2 from the [download page]({{ site.download_url }}). It contains the required files.
+Download a Flink package from the [download page]({{ site.download_url }}). It contains the required files.
 
 Extract the package using:
 
 {% highlight bash %}
-tar xvzf flink-{{ site.version }}-bin-hadoop2.tgz
+tar xvzf flink-{{ site.version }}-bin-scala*.tgz
 cd flink-{{site.version }}/
 {% endhighlight %}
 
+
 #### Start a Session
 
 Use the following command to start a session
@@ -125,7 +118,7 @@ If you don't want to change the configuration file to set configuration paramete
 
 The example invocation starts a single container for the ApplicationMaster which runs the Job Manager.
 
-The session cluster will automatically allocate additional containers which run the Task Managers when jobs are submitted to the cluster.
+The session cluster will automatically allocate additional containers which run the Task Managers when jobs are submitted to the cluster. 
 
 Once Flink is deployed in your YARN cluster, it will show you the connection details of the Job Manager.
 
@@ -338,4 +331,4 @@ The *JobManager* and AM are running in the same container. Once they successfull
 
 After that, the AM starts allocating the containers for Flink's TaskManagers, which will download the jar file and the modified configuration from the HDFS. Once these steps are completed, Flink is set up and ready to accept Jobs.
 
-{% top %}
+{% top %}
\ No newline at end of file


[flink] 05/08: [FLINK-11086][e2e] Use HADOOP_CLASSPATH in end to end tests

Posted by rm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rmetzger pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit c3648da4f4fdcd9c134ea05fd4316bc2146dd189
Author: Robert Metzger <rm...@apache.org>
AuthorDate: Mon May 4 10:16:44 2020 +0200

    [FLINK-11086][e2e] Use HADOOP_CLASSPATH in end to end tests
---
 flink-end-to-end-tests/pom.xml                     | 43 ++++++++++++++++++++++
 flink-end-to-end-tests/test-scripts/common.sh      | 10 +++++
 .../test-scripts/common_mesos_docker.sh            |  8 +++-
 .../docker-mesos-cluster/docker-compose.yml        |  1 +
 .../test_mesos_multiple_submissions.sh             |  4 +-
 .../test-scripts/test_mesos_wordcount.sh           |  4 +-
 .../test-scripts/test_streaming_bucketing.sh       |  2 +
 tools/verify_scala_suffixes.sh                     |  2 +-
 8 files changed, 70 insertions(+), 4 deletions(-)

diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml
index 3b97256..dbdcf49 100644
--- a/flink-end-to-end-tests/pom.xml
+++ b/flink-end-to-end-tests/pom.xml
@@ -91,6 +91,23 @@ under the License.
 		<module>flink-netty-shuffle-memory-control-test</module>
 	</modules>
 
+	<dependencies>
+		<dependency>
+			<!-- flink-yarn is required for getting the yarn classpath in the hadoop
+			bash end to end tests -->
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-yarn-tests</artifactId>
+			<version>${project.version}</version>
+			<scope>provided</scope>
+			<exclusions>
+				<exclusion>
+					<groupId>*</groupId>
+					<artifactId>*</artifactId>
+				</exclusion>
+			</exclusions>
+		</dependency>
+	</dependencies>
+
 	<profiles>
 		<profile>
 			<id>e2e-travis1</id>
@@ -154,6 +171,32 @@ under the License.
 	<build>
 		<plugins>
 			<plugin>
+				<artifactId>maven-resources-plugin</artifactId>
+				<!-- <version>3.1.0</version> -->
+				<executions>
+					<execution>
+						<id>copy-resources</id>
+						<!-- here the phase you need -->
+						<phase>package</phase>
+						<goals>
+							<goal>copy-resources</goal>
+						</goals>
+						<configuration>
+							<outputDirectory>${basedir}/test-scripts/hadoop</outputDirectory>
+							<resources>
+								<resource>
+									<directory>../flink-yarn-tests/target/</directory>
+									<filtering>true</filtering>
+									<includes>
+										<include>yarn.classpath</include>
+									</includes>
+								</resource>
+							</resources>
+						</configuration>
+					</execution>
+				</executions>
+			</plugin>
+			<plugin>
 				<groupId>org.apache.maven.plugins</groupId>
 				<artifactId>maven-deploy-plugin</artifactId>
 				<configuration>
diff --git a/flink-end-to-end-tests/test-scripts/common.sh b/flink-end-to-end-tests/test-scripts/common.sh
index 8329c12..c51fa18 100644
--- a/flink-end-to-end-tests/test-scripts/common.sh
+++ b/flink-end-to-end-tests/test-scripts/common.sh
@@ -55,6 +55,15 @@ REST_PROTOCOL="http"
 CURL_SSL_ARGS=""
 source "${TEST_INFRA_DIR}/common_ssl.sh"
 
+function set_hadoop_classpath {
+  YARN_CLASSPATH_LOCATION="${TEST_INFRA_DIR}/hadoop/yarn.classpath";
+  if [ ! -f $YARN_CLASSPATH_LOCATION ]; then
+    echo "File '$YARN_CLASSPATH_LOCATION' does not exist."
+    exit 1
+  fi
+  export HADOOP_CLASSPATH=`cat $YARN_CLASSPATH_LOCATION`
+}
+
 function print_mem_use_osx {
     declare -a mem_types=("active" "inactive" "wired down")
     used=""
@@ -353,6 +362,7 @@ function check_logs_for_errors {
       | grep -v "Failed Elasticsearch item request" \
       | grep -v "[Terror] modules" \
       | grep -v "HeapDumpOnOutOfMemoryError" \
+      | grep -v "error_prone_annotations" \
       | grep -ic "error" || true)
   if [[ ${error_count} -gt 0 ]]; then
     echo "Found error in log files:"
diff --git a/flink-end-to-end-tests/test-scripts/common_mesos_docker.sh b/flink-end-to-end-tests/test-scripts/common_mesos_docker.sh
index 5d90e5a..83aca21 100644
--- a/flink-end-to-end-tests/test-scripts/common_mesos_docker.sh
+++ b/flink-end-to-end-tests/test-scripts/common_mesos_docker.sh
@@ -44,8 +44,13 @@ function start_flink_cluster_with_mesos() {
         echo "ERROR: Could not build mesos image. Aborting..."
         exit 1
     fi
+    # build docker image with java and mesos
     build_image
 
+    # we need to export the MVN_REPO location so that mesos can access the files referenced in HADOOP_CLASSPATH
+    export MVN_REPO=`mvn help:evaluate -Dexpression=settings.localRepository -q -DforceStdout`
+
+    # start mesos cluster
     docker-compose -f $END_TO_END_DIR/test-scripts/docker-mesos-cluster/docker-compose.yml up -d
 
     # wait for the Mesos master and slave set up
@@ -58,7 +63,8 @@ function start_flink_cluster_with_mesos() {
     set_config_key "jobmanager.rpc.address" "mesos-master"
     set_config_key "rest.address" "mesos-master"
 
-    docker exec -itd mesos-master bash -c "${FLINK_DIR}/bin/mesos-appmaster.sh -Dmesos.master=mesos-master:5050"
+    docker exec --env HADOOP_CLASSPATH=$HADOOP_CLASSPATH -itd mesos-master bash -c "${FLINK_DIR}/bin/mesos-appmaster.sh -Dmesos.master=mesos-master:5050"
+
     wait_rest_endpoint_up "http://${NODENAME}:8081/taskmanagers" "Dispatcher" "\{\"taskmanagers\":\[.*\]\}"
     return 0
 }
diff --git a/flink-end-to-end-tests/test-scripts/docker-mesos-cluster/docker-compose.yml b/flink-end-to-end-tests/test-scripts/docker-mesos-cluster/docker-compose.yml
index 6445a3a..366d858 100644
--- a/flink-end-to-end-tests/test-scripts/docker-mesos-cluster/docker-compose.yml
+++ b/flink-end-to-end-tests/test-scripts/docker-mesos-cluster/docker-compose.yml
@@ -36,6 +36,7 @@ services:
     volumes:
       - ${END_TO_END_DIR}:${END_TO_END_DIR}
       - ${FLINK_DIR}:${FLINK_DIR}
+      - ${MVN_REPO}:${MVN_REPO}
     environment:
       MESOS_PORT: 5050
       MESOS_QUORUM: 1
diff --git a/flink-end-to-end-tests/test-scripts/test_mesos_multiple_submissions.sh b/flink-end-to-end-tests/test-scripts/test_mesos_multiple_submissions.sh
index fbfcd31..1cb0746 100755
--- a/flink-end-to-end-tests/test-scripts/test_mesos_multiple_submissions.sh
+++ b/flink-end-to-end-tests/test-scripts/test_mesos_multiple_submissions.sh
@@ -29,7 +29,7 @@ TEST_PROGRAM_JAR=$END_TO_END_DIR/flink-cli-test/target/PeriodicStreamingJob.jar
 
 function submit_job {
     local output_path=$1
-    docker exec mesos-master bash -c "${FLINK_DIR}/bin/flink run -d -p 1 ${TEST_PROGRAM_JAR} --durationInSecond ${DURATION} --outputPath ${output_path}" \
+    docker exec --env HADOOP_CLASSPATH=$HADOOP_CLASSPATH mesos-master bash -c "${FLINK_DIR}/bin/flink run -d -p 1 ${TEST_PROGRAM_JAR} --durationInSecond ${DURATION} --outputPath ${output_path}" \
         | grep "Job has been submitted with JobID" | sed 's/.* //g' | tr -d '\r'
 }
 
@@ -40,6 +40,8 @@ mkdir -p "${TEST_DATA_DIR}"
 # To ensure the old slots are being reused.
 set_config_key "mesos.resourcemanager.tasks.cpus" "${MESOS_AGENT_CPU}"
 
+set_hadoop_classpath
+
 start_flink_cluster_with_mesos
 
 JOB1_ID=$(submit_job ${FIRST_OUTPUT_LOCATION})
diff --git a/flink-end-to-end-tests/test-scripts/test_mesos_wordcount.sh b/flink-end-to-end-tests/test-scripts/test_mesos_wordcount.sh
index 94db94e..1e1c7c2 100755
--- a/flink-end-to-end-tests/test-scripts/test_mesos_wordcount.sh
+++ b/flink-end-to-end-tests/test-scripts/test_mesos_wordcount.sh
@@ -29,8 +29,10 @@ TEST_PROGRAM_JAR=${FLINK_DIR}/examples/batch/WordCount.jar
 
 mkdir -p "${TEST_DATA_DIR}"
 
+set_hadoop_classpath
+
 start_flink_cluster_with_mesos
 
-docker exec mesos-master nohup bash -c "${FLINK_DIR}/bin/flink run -p 1 ${TEST_PROGRAM_JAR} ${INPUT_ARGS} --output ${OUTPUT_LOCATION}"
+docker exec --env HADOOP_CLASSPATH=$HADOOP_CLASSPATH mesos-master nohup bash -c "${FLINK_DIR}/bin/flink run -p 1 ${TEST_PROGRAM_JAR} ${INPUT_ARGS} --output ${OUTPUT_LOCATION}"
 
 check_result_hash "Mesos WordCount test" "${OUTPUT_LOCATION}" "${RESULT_HASH}"
diff --git a/flink-end-to-end-tests/test-scripts/test_streaming_bucketing.sh b/flink-end-to-end-tests/test-scripts/test_streaming_bucketing.sh
index fbd4749..fef4dd2 100755
--- a/flink-end-to-end-tests/test-scripts/test_streaming_bucketing.sh
+++ b/flink-end-to-end-tests/test-scripts/test_streaming_bucketing.sh
@@ -19,6 +19,8 @@
 
 source "$(dirname "$0")"/common.sh
 
+set_hadoop_classpath
+
 TEST_PROGRAM_JAR=${END_TO_END_DIR}/flink-bucketing-sink-test/target/BucketingSinkTestProgram.jar
 JOB_OUTPUT_DIR=${TEST_DATA_DIR}/out/result
 LOG_DIR=${FLINK_DIR}/log
diff --git a/tools/verify_scala_suffixes.sh b/tools/verify_scala_suffixes.sh
index 714f6db..8fd6589 100755
--- a/tools/verify_scala_suffixes.sh
+++ b/tools/verify_scala_suffixes.sh
@@ -85,7 +85,7 @@ block_infected=0
 # b) exist only for dev purposes
 # c) no-one should depend on them
 e2e_modules=$(find flink-end-to-end-tests -mindepth 2 -maxdepth 5 -name 'pom.xml' -printf '%h\n' | sort -u | tr '\n' ',')
-excluded_modules=\!${e2e_modules//,/,\!},!flink-docs
+excluded_modules=\!${e2e_modules//,/,\!},!flink-docs,!flink-end-to-end-tests
 
 echo "Analyzing modules for Scala dependencies using 'mvn dependency:tree'."
 echo "If you haven't built the project, please do so first by running \"mvn clean install -DskipTests\""


[flink] 06/08: [FLINK-11086][AZP] Add Hadoop3 test profile to nightlies

Posted by rm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rmetzger pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 75c9cf12b107d1e94b16aa53548945b61acac7c5
Author: Robert Metzger <rm...@apache.org>
AuthorDate: Mon May 4 10:17:29 2020 +0200

    [FLINK-11086][AZP] Add Hadoop3 test profile to nightlies
---
 tools/azure-pipelines/build-apache-repo.yml | 11 +++++++++++
 1 file changed, 11 insertions(+)

diff --git a/tools/azure-pipelines/build-apache-repo.yml b/tools/azure-pipelines/build-apache-repo.yml
index f2d07bf..b00fc30 100644
--- a/tools/azure-pipelines/build-apache-repo.yml
+++ b/tools/azure-pipelines/build-apache-repo.yml
@@ -86,6 +86,17 @@ stages:
           jdk: jdk8
       - template: jobs-template.yml
         parameters:
+          stage_name: cron_hadoop313
+          test_pool_definition:
+            name: Default
+          e2e_pool_definition:
+            vmImage: 'ubuntu-16.04'
+          environment: PROFILE="-Dinclude-hadoop -Dinclude_hadoop_aws -Dhadoop.version=3.1.3 -Pe2e-hadoop -Phadoop3-tests"
+          run_end_to_end: true
+          container: flink-build-container
+          jdk: jdk8
+      - template: jobs-template.yml
+        parameters:
           stage_name: cron_scala212
           test_pool_definition:
             name: Default


[flink] 04/08: [FLINK-11086][yarn] Use YARN_APPLICATION_CLASSPATH instead of flink-shaded-hadoop fat jar in tests

Posted by rm...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rmetzger pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 082061da5ec52c7d4257adc272869c1ecb7fa222
Author: Robert Metzger <rm...@apache.org>
AuthorDate: Mon May 4 10:15:41 2020 +0200

    [FLINK-11086][yarn] Use YARN_APPLICATION_CLASSPATH instead of flink-shaded-hadoop fat jar in tests
---
 flink-yarn-tests/pom.xml                           | 76 +++++++++++++---------
 .../flink/yarn/YARNFileReplicationITCase.java      |  1 -
 .../flink/yarn/YARNHighAvailabilityITCase.java     |  5 --
 .../yarn/YARNSessionCapacitySchedulerITCase.java   | 10 +--
 .../apache/flink/yarn/YARNSessionFIFOITCase.java   |  5 --
 .../apache/flink/yarn/YarnConfigurationITCase.java |  1 -
 .../flink/yarn/YarnPrioritySchedulingITCase.java   |  1 -
 .../java/org/apache/flink/yarn/YarnTestBase.java   | 24 +++++--
 .../flink/yarn/YarnFileStageTestS3ITCase.java      |  5 ++
 9 files changed, 70 insertions(+), 58 deletions(-)

diff --git a/flink-yarn-tests/pom.xml b/flink-yarn-tests/pom.xml
index 2f5e9f4..a72a988 100644
--- a/flink-yarn-tests/pom.xml
+++ b/flink-yarn-tests/pom.xml
@@ -379,39 +379,53 @@ under the License.
 						<goals>
 							<goal>copy</goal>
 						</goals>
+						<configuration>
+							<artifactItems>
+								<artifactItem>
+									<groupId>org.apache.flink</groupId>
+									<artifactId>flink-examples-batch_${scala.binary.version}</artifactId>
+									<type>jar</type>
+									<classifier>WordCount</classifier>
+									<overWrite>true</overWrite>
+									<destFileName>BatchWordCount.jar</destFileName>
+								</artifactItem>
+								<artifactItem>
+									<groupId>org.apache.flink</groupId>
+									<artifactId>flink-examples-streaming_${scala.binary.version}</artifactId>
+									<type>jar</type>
+									<classifier>WordCount</classifier>
+									<overWrite>true</overWrite>
+									<destFileName>StreamingWordCount.jar</destFileName>
+								</artifactItem>
+								<artifactItem>
+									<groupId>org.apache.flink</groupId>
+									<artifactId>flink-examples-streaming_${scala.binary.version}</artifactId>
+									<type>jar</type>
+									<classifier>WindowJoin</classifier>
+									<overWrite>true</overWrite>
+									<destFileName>WindowJoin.jar</destFileName>
+								</artifactItem>
+							</artifactItems>
+							<outputDirectory>${project.build.directory}/programs</outputDirectory>
+							<overWriteReleases>false</overWriteReleases>
+							<overWriteSnapshots>true</overWriteSnapshots>
+						</configuration>
+					</execution>
+					<!-- Write classpath of flink-yarn to a file, so that the yarn tests can use it as their classpath
+						for the YARN "containers".
+					-->
+					<execution>
+						<id>store-classpath-in-target-for-tests</id>
+						<phase>package</phase>
+						<goals>
+							<goal>build-classpath</goal>
+						</goals>
+						<configuration>
+							<outputFile>${project.build.directory}/yarn.classpath</outputFile>
+							<excludeGroupIds>org.apache.flink</excludeGroupIds>
+						</configuration>
 					</execution>
 				</executions>
-				<configuration>
-					<artifactItems>
-						<artifactItem>
-							<groupId>org.apache.flink</groupId>
-							<artifactId>flink-examples-batch_${scala.binary.version}</artifactId>
-							<type>jar</type>
-							<classifier>WordCount</classifier>
-							<overWrite>true</overWrite>
-							<destFileName>BatchWordCount.jar</destFileName>
-						</artifactItem>
-						<artifactItem>
-							<groupId>org.apache.flink</groupId>
-							<artifactId>flink-examples-streaming_${scala.binary.version}</artifactId>
-							<type>jar</type>
-							<classifier>WordCount</classifier>
-							<overWrite>true</overWrite>
-							<destFileName>StreamingWordCount.jar</destFileName>
-						</artifactItem>
-						<artifactItem>
-							<groupId>org.apache.flink</groupId>
-							<artifactId>flink-examples-streaming_${scala.binary.version}</artifactId>
-							<type>jar</type>
-							<classifier>WindowJoin</classifier>
-							<overWrite>true</overWrite>
-							<destFileName>WindowJoin.jar</destFileName>
-						</artifactItem>
-					</artifactItems>
-					<outputDirectory>${project.build.directory}/programs</outputDirectory>
-					<overWriteReleases>false</overWriteReleases>
-					<overWriteSnapshots>true</overWriteSnapshots>
-				</configuration>
 			</plugin>
 		</plugins>
 	</build>
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNFileReplicationITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNFileReplicationITCase.java
index 993adbf..b2d6da5 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNFileReplicationITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNFileReplicationITCase.java
@@ -83,7 +83,6 @@ public class YARNFileReplicationITCase extends YarnTestBase {
 
 			yarnClusterDescriptor.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath()));
 			yarnClusterDescriptor.addShipFiles(Arrays.asList(flinkLibFolder.listFiles()));
-			yarnClusterDescriptor.addShipFiles(Arrays.asList(flinkShadedHadoopDir.listFiles()));
 
 			final int masterMemory = yarnClusterDescriptor.getFlinkConfiguration().get(JobManagerOptions.TOTAL_PROCESS_MEMORY).getMebiBytes();
 			final ClusterSpecification clusterSpecification = new ClusterSpecification.ClusterSpecificationBuilder()
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
index a8b799a..b3a9269 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java
@@ -74,7 +74,6 @@ import javax.annotation.Nonnull;
 import java.io.File;
 import java.io.IOException;
 import java.time.Duration;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
@@ -160,8 +159,6 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
 				OperatingSystem.isLinux() || OperatingSystem.isMac() || OperatingSystem.isFreeBSD() || OperatingSystem.isSolaris());
 
 			final YarnClusterDescriptor yarnClusterDescriptor = setupYarnClusterDescriptor();
-			yarnClusterDescriptor.addShipFiles(Arrays.asList(flinkShadedHadoopDir.listFiles()));
-
 			final RestClusterClient<ApplicationId> restClusterClient = deploySessionCluster(yarnClusterDescriptor);
 
 			try {
@@ -186,8 +183,6 @@ public class YARNHighAvailabilityITCase extends YarnTestBase {
 	public void testJobRecoversAfterKillingTaskManager() throws Exception {
 		runTest(() -> {
 			final YarnClusterDescriptor yarnClusterDescriptor = setupYarnClusterDescriptor();
-			yarnClusterDescriptor.addShipFiles(Arrays.asList(flinkShadedHadoopDir.listFiles()));
-
 			final RestClusterClient<ApplicationId> restClusterClient = deploySessionCluster(yarnClusterDescriptor);
 			try {
 				final JobID jobId = submitJob(restClusterClient);
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
index c8b6280..490c858 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java
@@ -155,9 +155,7 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 	public void testStartYarnSessionClusterInQaTeamQueue() throws Exception {
 		runTest(() -> runWithArgs(new String[]{
 				"-j", flinkUberjar.getAbsolutePath(),
-				"-t", flinkLibFolder.getAbsolutePath(),
-				"-t", flinkShadedHadoopDir.getAbsolutePath(),
-				"-jm", "768m",
+				"-t", flinkLibFolder.getAbsolutePath(), "-jm", "768m",
 				"-tm", "1024m", "-qu", "qa-team"},
 			"JobManager Web Interface:", null, RunTypes.YARN_SESSION, 0));
 	}
@@ -177,7 +175,6 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 			runWithArgs(new String[]{"run", "-m", "yarn-cluster",
 					"-yj", flinkUberjar.getAbsolutePath(),
 					"-yt", flinkLibFolder.getAbsolutePath(),
-					"-yt", flinkShadedHadoopDir.getAbsolutePath(),
 					"-ys", "2", //test that the job is executed with a DOP of 2
 					"-yjm", "768m",
 					"-ytm", "1024m", exampleJarLocation.getAbsolutePath()},
@@ -213,7 +210,6 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 			runWithArgs(new String[]{"run", "-m", "yarn-cluster",
 					"-yj", flinkUberjar.getAbsolutePath(),
 					"-yt", flinkLibFolder.getAbsolutePath(),
-					"-yt", flinkShadedHadoopDir.getAbsolutePath(),
 					"-ys", "2", //test that the job is executed with a DOP of 2
 					"-yjm", "768m",
 					"-ytm", taskManagerMemoryMB + "m",
@@ -251,7 +247,6 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 			final Runner yarnSessionClusterRunner = startWithArgs(new String[]{
 					"-j", flinkUberjar.getAbsolutePath(),
 					"-t", flinkLibFolder.getAbsolutePath(),
-					"-t", flinkShadedHadoopDir.getAbsolutePath(),
 					"-jm", "768m",
 					"-tm", "1024m",
 					"-s", "3", // set the slots 3 to check if the vCores are set properly!
@@ -393,7 +388,6 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 			try {
 				runWithArgs(new String[]{"-j", flinkUberjar.getAbsolutePath(),
 					"-t", flinkLibFolder.getAbsolutePath(),
-					"-t", flinkShadedHadoopDir.getAbsolutePath(),
 					"-jm", "768m",
 					"-tm", "1024m",
 					"-qu", "doesntExist"}, "to unknown queue: doesntExist", null, RunTypes.YARN_SESSION, 1);
@@ -420,7 +414,6 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 					"-m", "yarn-cluster",
 					"-yj", flinkUberjar.getAbsolutePath(),
 					"-yt", flinkLibFolder.getAbsolutePath(),
-					"-yt", flinkShadedHadoopDir.getAbsolutePath(),
 					"-ys", "2",
 					"-yjm", "768m",
 					"-ytm", "1024m", exampleJarLocation.getAbsolutePath()},
@@ -495,7 +488,6 @@ public class YARNSessionCapacitySchedulerITCase extends YarnTestBase {
 				"run", "-m", "yarn-cluster",
 				"-yj", flinkUberjar.getAbsolutePath(),
 				"-yt", flinkLibFolder.getAbsolutePath(),
-				"-yt", flinkShadedHadoopDir.getAbsolutePath(),
 				"-yjm", "768m",
 				"-yD", YarnConfigOptions.APPLICATION_TAGS.key() + "=test-tag",
 				"-ytm", "1024m",
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
index fbff39d..bb1d98b 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java
@@ -107,9 +107,6 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 			args.add("-t");
 			args.add(flinkLibFolder.getAbsolutePath());
 
-			args.add("-t");
-			args.add(flinkShadedHadoopDir.getAbsolutePath());
-
 			args.add("-jm");
 			args.add("768m");
 
@@ -247,7 +244,6 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 			runWithArgs(new String[]{
 				"-j", flinkUberjar.getAbsolutePath(),
 				"-t", flinkLibFolder.getAbsolutePath(),
-				"-t", flinkShadedHadoopDir.getAbsolutePath(),
 				"-jm", "256m",
 				"-tm", "1585m"}, "Number of connected TaskManagers changed to", null, RunTypes.YARN_SESSION, 0);
 			LOG.info("Finished testResourceComputation()");
@@ -280,7 +276,6 @@ public class YARNSessionFIFOITCase extends YarnTestBase {
 			runWithArgs(new String[]{
 				"-j", flinkUberjar.getAbsolutePath(),
 				"-t", flinkLibFolder.getAbsolutePath(),
-				"-t", flinkShadedHadoopDir.getAbsolutePath(),
 				"-jm", "256m",
 				"-tm", "3840m"}, "Number of connected TaskManagers changed to", null, RunTypes.YARN_SESSION, 0);
 			LOG.info("Finished testfullAlloc()");
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java
index 900abb6..cceb3a0 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java
@@ -102,7 +102,6 @@ public class YarnConfigurationITCase extends YarnTestBase {
 
 			clusterDescriptor.setLocalJarPath(new Path(flinkUberjar.getAbsolutePath()));
 			clusterDescriptor.addShipFiles(Arrays.asList(flinkLibFolder.listFiles()));
-			clusterDescriptor.addShipFiles(Arrays.asList(flinkShadedHadoopDir.listFiles()));
 
 			final File streamingWordCountFile = getTestJarPath("WindowJoin.jar");
 
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnPrioritySchedulingITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnPrioritySchedulingITCase.java
index 280cd01..16fd866 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnPrioritySchedulingITCase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnPrioritySchedulingITCase.java
@@ -52,7 +52,6 @@ public class YarnPrioritySchedulingITCase extends YarnTestBase {
 			final Runner yarnSessionClusterRunner = startWithArgs(new String[]{
 					"-j", flinkUberjar.getAbsolutePath(),
 					"-t", flinkLibFolder.getAbsolutePath(),
-					"-t", flinkShadedHadoopDir.getAbsolutePath(),
 					"-jm", "768m",
 					"-tm", "1024m",
 					"-Dyarn.application.priority=" + priority},
diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
index 5334e93..003c7f7 100644
--- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
+++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnTestBase.java
@@ -74,7 +74,6 @@ import java.io.InputStream;
 import java.io.PipedInputStream;
 import java.io.PipedOutputStream;
 import java.io.PrintStream;
-import java.nio.file.Paths;
 import java.time.Duration;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -172,7 +171,6 @@ public abstract class YarnTestBase extends TestLogger {
 	 * Temporary folder where Flink configurations will be kept for secure run.
 	 */
 	protected static File tempConfPathForSecureRun = null;
-	protected static File flinkShadedHadoopDir;
 
 	protected static File yarnSiteXML = null;
 	protected static File hdfsSiteXML = null;
@@ -197,6 +195,24 @@ public abstract class YarnTestBase extends TestLogger {
 		// so we have to change the number of cores for testing.
 		YARN_CONFIGURATION.setInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 20000); // 20 seconds expiry (to ensure we properly heartbeat with YARN).
 		YARN_CONFIGURATION.setFloat(YarnConfiguration.NM_MAX_PER_DISK_UTILIZATION_PERCENTAGE, 99.0F);
+
+		YARN_CONFIGURATION.set(YarnConfiguration.YARN_APPLICATION_CLASSPATH, getYarnClasspath());
+	}
+
+	/**
+	 * Searches for the yarn.classpath file generated by the "dependency:build-classpath" maven plugin in
+	 * "flink-yarn".
+	 * @return a classpath suitable for running all YARN-launched JVMs
+	 */
+	private static String getYarnClasspath() {
+		final String start = "../flink-yarn-tests";
+		try {
+			File classPathFile = findFile(start, (dir, name) -> name.equals("yarn.classpath"));
+			return FileUtils.readFileToString(classPathFile); // potential NPE is supposed to be fatal
+		} catch (Throwable t) {
+			LOG.error("Error while getting YARN classpath in {}", new File(start).getAbsoluteFile(), t);
+			throw new RuntimeException("Error while getting YARN classpath", t);
+		}
 	}
 
 	public static void populateYarnSecureConfigurations(Configuration conf, String principal, String keytab) {
@@ -431,7 +447,7 @@ public abstract class YarnTestBase extends TestLogger {
 							if (!whitelistedFound) {
 								// logging in FATAL to see the actual message in TRAVIS tests.
 								Marker fatal = MarkerFactory.getMarker("FATAL");
-								LOG.error(fatal, "Prohibited String '{}' in line '{}'", aProhibited, lineFromFile);
+								LOG.error(fatal, "Prohibited String '{}' in '{}:{}'", aProhibited, f.getAbsolutePath(), lineFromFile);
 
 								StringBuilder logExcerpt = new StringBuilder();
 
@@ -637,8 +653,6 @@ public abstract class YarnTestBase extends TestLogger {
 		Assert.assertNotNull("Flink uberjar not found", flinkUberjar);
 		String flinkDistRootDir = flinkUberjar.getParentFile().getParent();
 		flinkLibFolder = flinkUberjar.getParentFile(); // the uberjar is located in lib/
-		// the hadoop jar was copied into the target/shaded-hadoop directory during the build
-		flinkShadedHadoopDir = Paths.get("target/shaded-hadoop").toFile();
 		Assert.assertNotNull("Flink flinkLibFolder not found", flinkLibFolder);
 		Assert.assertTrue("lib folder not found", flinkLibFolder.exists());
 		Assert.assertTrue("lib folder not found", flinkLibFolder.isDirectory());
diff --git a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java
index 11d48bd..bfde0c9 100644
--- a/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java
+++ b/flink-yarn/src/test/java/org/apache/flink/yarn/YarnFileStageTestS3ITCase.java
@@ -29,7 +29,9 @@ import org.apache.flink.testutils.junit.RetryRule;
 import org.apache.flink.testutils.s3.S3TestCredentials;
 import org.apache.flink.util.TestLogger;
 
+import org.apache.hadoop.util.VersionUtil;
 import org.junit.AfterClass;
+import org.junit.Assume;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Rule;
@@ -174,6 +176,9 @@ public class YarnFileStageTestS3ITCase extends TestLogger {
 	@Test
 	@RetryOnFailure(times = 3)
 	public void testRecursiveUploadForYarnS3n() throws Exception {
+		// skip test on Hadoop 3: https://issues.apache.org/jira/browse/HADOOP-14738
+		Assume.assumeTrue("This test is skipped for Hadoop versions above 3", VersionUtil.compareVersions(System.getProperty("hadoop.version"), "3.0.0") < 0);
+
 		try {
 			Class.forName("org.apache.hadoop.fs.s3native.NativeS3FileSystem");
 		} catch (ClassNotFoundException e) {