You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/10/26 16:25:45 UTC

[GitHub] [flink] rmetzger opened a new pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

rmetzger opened a new pull request #13796:
URL: https://github.com/apache/flink/pull/13796


   
   ## What is the purpose of the change
   
   For every release, we are manually validating the NOTICE files, according to this wiki page: https://cwiki.apache.org/confluence/display/FLINK/Licensing
   
   The most time-consuming task is ensuring that all modules that deploy a shaded dependency to maven central are properly documenting this dependency in their NOTICE file.
   
   I would like to add a tool to Flink that checks if all shaded dependencies are at least mentioned in the NOTICE file.
   We will still need to perform a manual checks, but the tool should catch the most common, severe and difficult to find problems.
   
   
   ## Verifying this change
   
   The CI build should fail with a number of current licensing issues present on `master`. I haven't analyzed all of them to the fullest extend. Once the principle idea of this PR is approved, I will fix them.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-717170059


   I went through the issues the checker found, but most of them appear to be invalid:
   
   Contradicting statement(similar instances exist for kinesis, flink-oss-fs-hadoop, and possibly others):
   
   ```
   Could not find dependency org.apache.commons:commons-compress:1.20 in NOTICE file /__w/2/s/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE
   Dependency com.apache.commons:commons-compress:1.20 is mentioned in NOTICE file /__w/2/s/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE, but is not expected there
   ```
   
   The flink-dist warnings appears to be mostly incorrect.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol edited a comment on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol edited a comment on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-718130365


   There is no benefit in having the generated files in git, it could just be step of the release process.
   
   On CI we would compile Flink and pipe the build output into a file, then run the generator with this file as input to ensure that the NOTICE files can be generated.
   
   > I've quickly looked at building our own shade resource transformer
   
   FYI: Last time I tried it wasn't possible to write custom transformers (it _looks_ like you can, but I always had classpath issues), and even if you could it would interfere with other transformers touching the notice file (like the appender).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517244148



##########
File path: tools/ci/java-ci-tools/pom.xml
##########
@@ -23,49 +23,32 @@ under the License.
 	<modelVersion>4.0.0</modelVersion>
 
 	<parent>
-		<groupId>org.apache</groupId>
-		<artifactId>apache</artifactId>
-		<version>18</version>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-parent</artifactId>
+		<version>1.12-SNAPSHOT</version>
 	</parent>
 
-	<groupId>org.apache.flink</groupId>
 	<artifactId>java-ci-tools</artifactId>
 	<version>1.12-SNAPSHOT</version>
 	<name>Flink : Tools : CI : Java</name>
 
-	<packaging>jar</packaging>
-	<properties>
-		<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-		<maven.compiler.source>1.8</maven.compiler.source>
-		<maven.compiler.target>1.8</maven.compiler.target>
-		<log4j.version>2.12.1</log4j.version>
-	</properties>
-
 	<dependencies>
 		<dependency>
 			<groupId>com.google.guava</groupId>
 			<artifactId>guava</artifactId>
 			<version>30.0-jre</version>
 		</dependency>
-		<dependency>
-			<groupId>org.slf4j</groupId>
-			<artifactId>slf4j-api</artifactId>
-			<version>1.7.30</version>
-		</dependency>
 		<dependency>
 			<groupId>org.apache.logging.log4j</groupId>
 			<artifactId>log4j-slf4j-impl</artifactId>
-			<version>${log4j.version}</version>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.logging.log4j</groupId>
 			<artifactId>log4j-api</artifactId>
-			<version>${log4j.version}</version>

Review comment:
       this wasn't addressed?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-716686693






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger closed pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger closed pull request #13796:
URL: https://github.com/apache/flink/pull/13796


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-716686693


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e58da69e4aeb75fddcdb08c1ae89c2bc9789902a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e58da69e4aeb75fddcdb08c1ae89c2bc9789902a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e58da69e4aeb75fddcdb08c1ae89c2bc9789902a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-716686693


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e58da69e4aeb75fddcdb08c1ae89c2bc9789902a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8322",
       "triggerID" : "e58da69e4aeb75fddcdb08c1ae89c2bc9789902a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54cc6f4d9ff39c498698be95e0640ab9b778b11f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8685",
       "triggerID" : "54cc6f4d9ff39c498698be95e0640ab9b778b11f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be6463deec21201f41db750bf996f98723868b64",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8846",
       "triggerID" : "be6463deec21201f41db750bf996f98723868b64",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bf8c40adfc624f0683720d0b6a31283b9962cdaa",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8982",
       "triggerID" : "bf8c40adfc624f0683720d0b6a31283b9962cdaa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94b6fa9e2a018f95b88b5b7ba1146103a94d313",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8996",
       "triggerID" : "e94b6fa9e2a018f95b88b5b7ba1146103a94d313",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6256df86eba2c9a54dc26c8d32c20d6399bfc1f3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9002",
       "triggerID" : "6256df86eba2c9a54dc26c8d32c20d6399bfc1f3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bf8c40adfc624f0683720d0b6a31283b9962cdaa Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8982) 
   * e94b6fa9e2a018f95b88b5b7ba1146103a94d313 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8996) 
   * 6256df86eba2c9a54dc26c8d32c20d6399bfc1f3 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9002) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517181365



##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -251,21 +266,19 @@ private static String readFile(Path path) throws IOException {
 	}
 
 	private static List<String> loadFromResources(String fileName) {
-		List<String> res = new ArrayList<>();
-		try (InputStream in = LicenseChecker.class.getResourceAsStream("/" + fileName)) {
-			try (Scanner scanner = new Scanner(in)) {
-				while (scanner.hasNext()) {
-					String line = scanner.nextLine();
-					if (!line.startsWith("#")){
-						res.add(line);
-					}
-				}
-			}
-		} catch (IOException e) {
+		try {
+			Path resource = Paths.get(LicenseChecker.class.getResource("/" + fileName).toURI());
+			List<String> result = Files
+				.readAllLines(resource)
+				.stream()
+				.filter(line -> !line.startsWith("#") && !line.isEmpty())
+				.collect(Collectors.toList());
+			LOG.debug("Loaded {} items from resource {}", result.size(), fileName);
+			return result;
+		} catch (Throwable e) {
 			LOG.warn("Error while loading resource", e);

Review comment:
       I didn't want to throw an exception here, because it would lead to an ugly `ExceptionInInitializerError`, but you are right, it is probably worth it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517177573



##########
File path: flink-connectors/flink-sql-connector-hbase-1.4/pom.xml
##########
@@ -87,6 +87,7 @@ under the License.
 									<exclude>org.apache.hbase:hbase-metrics*</exclude>
 									<exclude>org.apache.hbase:hbase-server*</exclude>
 									<exclude>org.apache.hbase:hbase-hadoop*-compat</exclude>
+									<exclude>org.apache.hbase:hbase-common:jar:tests</exclude>

Review comment:
       This is the dependency tree:
   ```
   [INFO] org.apache.flink:flink-sql-connector-hbase-1.4_2.11:jar:1.12-SNAPSHOT
   [INFO] +- org.apache.flink:flink-connector-hbase-1.4_2.11:jar:1.12-SNAPSHOT:compile
   [INFO] |  \- org.apache.flink:flink-connector-hbase-base_2.11:jar:1.12-SNAPSHOT:compile
   [INFO] |     +- org.apache.hbase:hbase-server:jar:1.4.3:compile
   [INFO] |     |  +- org.apache.hbase:hbase-common:jar:1.4.3:compile
   [INFO] |     |  |  +- org.apache.avro:avro:jar:1.10.0:compile
   [INFO] |     |  |  |  +- com.fasterxml.jackson.core:jackson-core:jar:2.10.1:compile
   [INFO] |     |  |  |  \- com.fasterxml.jackson.core:jackson-databind:jar:2.10.1:compile
   [INFO] |     |  |  |     \- com.fasterxml.jackson.core:jackson-annotations:jar:2.10.1:compile
   [INFO] |     |  |  \- com.github.stephenc.findbugs:findbugs-annotations:jar:1.3.9-1:compile
   [INFO] |     |  +- org.apache.hbase:hbase-protocol:jar:1.4.3:compile
   [INFO] |     |  +- org.apache.hbase:hbase-procedure:jar:1.4.3:compile
   [INFO] |     |  |  \- org.apache.hbase:hbase-common:jar:tests:1.4.3:compile
   ```
   
   I can not add an exclude on hbase-server, because I can not exclude based on the `tests` classifier.
   
   The other question is: why are we depending on `hbase-server`. It would probably be fine to depend on `hbase-common` and `hbase-client` (just judging from the names). I quickly tried that, but it lead to all kinds of problems with the tests.
   How about working with the shade exclude for now, but filing a ticket, to solve this properly in the next release cycle?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517211912



##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+//CHECKSTYLE.OFF: regexp|imports
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+//CHECKSTYLE.ON: regexp|imports
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), Paths.get(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:[^:]+:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	// Examples:
+	// "- org.apache.htrace:htrace-core:3.1.0-incubating"
+	// or
+	// "This project bundles "net.jcip:jcip-annotations:1.0".
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, Path root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithBundledDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithBundledDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithBundledDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<Path> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithBundledDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (Path noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithBundledDependencies, noticeFile);
+		}
+
+		return severeIssueCount;
+	}
+
+	private static int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<Path> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(Path noticeFile) {
+		Path moduleDirectory = noticeFile.getParent() // META-INF
+			.getParent() // resources
+			.getParent() // main
+			.getParent() // src
+			.getParent(); // <-- module name
+		return moduleDirectory.getFileName().toString();
+	}
+
+	private static int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, Path noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		List<String> noticeContents = Files.readAllLines(noticeFile);
+
+		if (noticeContents.isEmpty()) {
+			LOG.error("Notice file empty {}", noticeFile);
+			severeIssueCount++;
+		}
+
+		// first line must be the module name.
+		if (!noticeContents.get(0).equals(moduleName)) {
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, noticeContents.get(0));
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		for (String line : noticeContents) {
+			Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+			if (noticeDependencyMatcher.find()) {
+				String groupId = noticeDependencyMatcher.group(1);
+				String artifactId = noticeDependencyMatcher.group(2);
+				String version = noticeDependencyMatcher.group(3);
+				if (groupId == null && artifactId == null && version == null) { // "bundles" case
+					groupId = noticeDependencyMatcher.group(5);
+					artifactId = noticeDependencyMatcher.group(6);
+					version = noticeDependencyMatcher.group(7);
+				}
+				IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+				if (!declaredDependencies.add(toAdd)) {
+					LOG.error("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					severeIssueCount++;
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Collection<IncludedDependency> expectedDependencies = modulesWithShadedDependencies.get(moduleName);
+		for (IncludedDependency expectedDependency : expectedDependencies) {
+			if (!declaredDependencies.contains(expectedDependency)) {
+				LOG.error("Could not find dependency {} in NOTICE file {}", expectedDependency, noticeFile);
+				severeIssueCount++;
+			}
+		}
+
+		boolean moduleDefinesExcessDependencies = MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName);
+
+		// print all dependencies defined in NOTICE file, which were not expected
+		for (IncludedDependency declaredDependency : declaredDependencies) {
+			if (!expectedDependencies.contains(declaredDependency)) {
+				if (moduleDefinesExcessDependencies) {
+					LOG.debug(
+						"Dependency {} is mentioned in NOTICE file {}, but was not mentioned by the build output as a bundled dependency",
+						declaredDependency,
+						noticeFile);
+				} else {
+					LOG.warn(
+						"Dependency {} is mentioned in NOTICE file {}, but is not expected there",
+						declaredDependency,
+						noticeFile);
+				}
+
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static List<Path> findNoticeFiles(Path root) throws IOException {
+		return Files.walk(root)
+			.filter(file -> {
+				int nameCount = file.getNameCount();
+				return file.getName(nameCount - 3).toString().equals("resources")
+					&& file.getName(nameCount - 2).toString().equals("META-INF")
+					&& file.getName(nameCount - 1).toString().equals("NOTICE");
+			})
+			.collect(Collectors.toList());
+	}
+
+	private static Multimap<String, IncludedDependency> parseModulesFromBuildResult(File buildResult) throws IOException {
+		Multimap<String, IncludedDependency> result = ArrayListMultimap.create();
+
+		try (Stream<String> lines = Files.lines(buildResult.toPath())) {
+			//String line;
+			String currentShadeModule = null;
+			String currentDependencyCopyModule = null;
+			for (String line : (Iterable<String>) lines::iterator) {
+				Matcher nextShadeModuleMatcher = SHADE_NEXT_MODULE_PATTERN.matcher(line);
+				if (nextShadeModuleMatcher.find()) {
+					currentShadeModule = nextShadeModuleMatcher.group(2);
+				}
+
+				Matcher nextDependencyCopyModuleMatcher = DEPENDENCY_COPY_NEXT_MODULE_PATTERN.matcher(line);
+				if (nextDependencyCopyModuleMatcher.find()) {
+					currentDependencyCopyModule = nextDependencyCopyModuleMatcher.group(1);
+				}
+
+				if (currentShadeModule != null) {
+					Matcher includeMatcher = SHADE_INCLUDE_MODULE_PATTERN.matcher(line);
+					if (includeMatcher.find()) {
+						String groupId = includeMatcher.group(1);
+						String artifactId = includeMatcher.group(2);
+						String version = includeMatcher.group(3);
+						if (!"org.apache.flink".equals(groupId)) {
+							result.put(currentShadeModule, IncludedDependency.create(groupId, artifactId, version));
+						}
+					}
+				}
+
+				if (currentDependencyCopyModule != null) {
+					Matcher copyMatcher = DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN.matcher(line);
+					if (copyMatcher.find()) {
+						String groupId = copyMatcher.group(1);
+						String artifactId = copyMatcher.group(2);
+						String version = copyMatcher.group(3);
+						if (!"org.apache.flink".equals(groupId)) {
+							result.put(currentDependencyCopyModule, IncludedDependency.create(groupId, artifactId, version));
+						}
+					}
+				}
+				if (line.contains("Replacing original artifact with shaded artifact")) {
+					currentShadeModule = null;
+				}
+				if (line.contains("Copying")) {
+					currentDependencyCopyModule = null;
+				}
+			}

Review comment:
       continuation of [the above comment](https://github.com/apache/flink/pull/13796/files?file-filters%5B%5D=.activation&file-filters%5B%5D=.java&file-filters%5B%5D=.modulelist&file-filters%5B%5D=.properties&file-filters%5B%5D=.sh&file-filters%5B%5D=.yml&file-filters%5B%5D=No+extension#r517211857):
   ```suggestion
   			});
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-717859165


   Once I'm done with the license check for this release, I can make an assessment of how well the tool works.
   If I'm confident that it can also handle edge-cases, I could look into automating it. What we need is basically a mapping from dependency --> license. We could make the build fail if this mapping doesn't exist for a dependency.
   
   I will proceed with addressing the licensing issues I've found so far. I was planning to open separate PRs for different areas of Flink (python, table, connectors), or shall I add them all here to this PR? (this would be easier for me, but only works if you are willing to review them all at once)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-716686693


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e58da69e4aeb75fddcdb08c1ae89c2bc9789902a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8322",
       "triggerID" : "e58da69e4aeb75fddcdb08c1ae89c2bc9789902a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54cc6f4d9ff39c498698be95e0640ab9b778b11f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8685",
       "triggerID" : "54cc6f4d9ff39c498698be95e0640ab9b778b11f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be6463deec21201f41db750bf996f98723868b64",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8846",
       "triggerID" : "be6463deec21201f41db750bf996f98723868b64",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bf8c40adfc624f0683720d0b6a31283b9962cdaa",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8982",
       "triggerID" : "bf8c40adfc624f0683720d0b6a31283b9962cdaa",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * be6463deec21201f41db750bf996f98723868b64 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8846) 
   * bf8c40adfc624f0683720d0b6a31283b9962cdaa Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8982) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517232472



##########
File path: flink-connectors/flink-sql-connector-hbase-1.4/pom.xml
##########
@@ -87,6 +87,7 @@ under the License.
 									<exclude>org.apache.hbase:hbase-metrics*</exclude>
 									<exclude>org.apache.hbase:hbase-server*</exclude>
 									<exclude>org.apache.hbase:hbase-hadoop*-compat</exclude>
+									<exclude>org.apache.hbase:hbase-common:jar:tests</exclude>

Review comment:
       As for whether we should depend on hbase-server, it may make sense to look into it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517254711



##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+//CHECKSTYLE.OFF: regexp|imports
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+//CHECKSTYLE.ON: regexp|imports
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), Paths.get(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:[^:]+:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	// Examples:
+	// "- org.apache.htrace:htrace-core:3.1.0-incubating"
+	// or
+	// "This project bundles "net.jcip:jcip-annotations:1.0".
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, Path root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithBundledDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithBundledDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithBundledDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<Path> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithBundledDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (Path noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithBundledDependencies, noticeFile);
+		}
+
+		return severeIssueCount;
+	}
+
+	private static int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<Path> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(Path noticeFile) {
+		Path moduleDirectory = noticeFile.getParent() // META-INF
+			.getParent() // resources
+			.getParent() // main
+			.getParent() // src
+			.getParent(); // <-- module name
+		return moduleDirectory.getFileName().toString();
+	}
+
+	private static int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, Path noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		List<String> noticeContents = Files.readAllLines(noticeFile);
+
+		if (noticeContents.isEmpty()) {
+			LOG.error("Notice file empty {}", noticeFile);
+			severeIssueCount++;
+		}
+
+		// first line must be the module name.
+		if (!noticeContents.get(0).equals(moduleName)) {
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, noticeContents.get(0));
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		for (String line : noticeContents) {
+			Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+			if (noticeDependencyMatcher.find()) {
+				String groupId = noticeDependencyMatcher.group(1);
+				String artifactId = noticeDependencyMatcher.group(2);
+				String version = noticeDependencyMatcher.group(3);
+				if (groupId == null && artifactId == null && version == null) { // "bundles" case
+					groupId = noticeDependencyMatcher.group(5);
+					artifactId = noticeDependencyMatcher.group(6);
+					version = noticeDependencyMatcher.group(7);
+				}
+				IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+				if (!declaredDependencies.add(toAdd)) {
+					LOG.error("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					severeIssueCount++;
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Collection<IncludedDependency> expectedDependencies = modulesWithShadedDependencies.get(moduleName);
+		for (IncludedDependency expectedDependency : expectedDependencies) {
+			if (!declaredDependencies.contains(expectedDependency)) {
+				LOG.error("Could not find dependency {} in NOTICE file {}", expectedDependency, noticeFile);
+				severeIssueCount++;
+			}
+		}
+
+		boolean moduleDefinesExcessDependencies = MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName);
+
+		// print all dependencies defined in NOTICE file, which were not expected
+		for (IncludedDependency declaredDependency : declaredDependencies) {
+			if (!expectedDependencies.contains(declaredDependency)) {
+				if (moduleDefinesExcessDependencies) {
+					LOG.debug(
+						"Dependency {} is mentioned in NOTICE file {}, but was not mentioned by the build output as a bundled dependency",
+						declaredDependency,
+						noticeFile);
+				} else {
+					LOG.warn(
+						"Dependency {} is mentioned in NOTICE file {}, but is not expected there",
+						declaredDependency,
+						noticeFile);
+				}
+
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static List<Path> findNoticeFiles(Path root) throws IOException {
+		return Files.walk(root)
+			.filter(file -> {
+				int nameCount = file.getNameCount();
+				return file.getName(nameCount - 3).toString().equals("resources")
+					&& file.getName(nameCount - 2).toString().equals("META-INF")
+					&& file.getName(nameCount - 1).toString().equals("NOTICE");
+			})
+			.collect(Collectors.toList());
+	}
+
+	private static Multimap<String, IncludedDependency> parseModulesFromBuildResult(File buildResult) throws IOException {
+		Multimap<String, IncludedDependency> result = ArrayListMultimap.create();
+
+		try (Stream<String> lines = Files.lines(buildResult.toPath())) {
+			//String line;
+			String currentShadeModule = null;
+			String currentDependencyCopyModule = null;
+			for (String line : (Iterable<String>) lines::iterator) {

Review comment:
       damn it. I just dislike the cast.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r516602034



##########
File path: tools/ci/java-ci-tools/pom.xml
##########
@@ -23,49 +23,32 @@ under the License.
 	<modelVersion>4.0.0</modelVersion>
 
 	<parent>
-		<groupId>org.apache</groupId>
-		<artifactId>apache</artifactId>
-		<version>18</version>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-parent</artifactId>
+		<version>1.12-SNAPSHOT</version>
 	</parent>
 
-	<groupId>org.apache.flink</groupId>
 	<artifactId>java-ci-tools</artifactId>
 	<version>1.12-SNAPSHOT</version>
 	<name>Flink : Tools : CI : Java</name>
 
-	<packaging>jar</packaging>
-	<properties>
-		<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-		<maven.compiler.source>1.8</maven.compiler.source>
-		<maven.compiler.target>1.8</maven.compiler.target>
-		<log4j.version>2.12.1</log4j.version>
-	</properties>
-
 	<dependencies>
 		<dependency>
 			<groupId>com.google.guava</groupId>
 			<artifactId>guava</artifactId>
 			<version>30.0-jre</version>
 		</dependency>
-		<dependency>
-			<groupId>org.slf4j</groupId>
-			<artifactId>slf4j-api</artifactId>
-			<version>1.7.30</version>
-		</dependency>
 		<dependency>
 			<groupId>org.apache.logging.log4j</groupId>
 			<artifactId>log4j-slf4j-impl</artifactId>
-			<version>${log4j.version}</version>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.logging.log4j</groupId>
 			<artifactId>log4j-api</artifactId>
-			<version>${log4j.version}</version>

Review comment:
       Don't we have to explicitly set these as compile dependencies?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-718289508


   > One benefit would be, that our release process is less complicated
   
   Doesn't really apply imo because this can be easily automated. While the (build) process is more complicated, it is not more complicated for the RM.
   
   > Second benefit is that it is easier to check that the generated NOTICE files are valid when you have to regenerate and commit them. With your approach, we would wait with that until the release, and it would only be done by a few "experts".
   
   Realistically, only experts will look at the results _anyway_. Once something is generated people stop caring about correctness, and just go through the motions.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-716686693


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e58da69e4aeb75fddcdb08c1ae89c2bc9789902a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8322",
       "triggerID" : "e58da69e4aeb75fddcdb08c1ae89c2bc9789902a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54cc6f4d9ff39c498698be95e0640ab9b778b11f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8685",
       "triggerID" : "54cc6f4d9ff39c498698be95e0640ab9b778b11f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be6463deec21201f41db750bf996f98723868b64",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8846",
       "triggerID" : "be6463deec21201f41db750bf996f98723868b64",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bf8c40adfc624f0683720d0b6a31283b9962cdaa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8982",
       "triggerID" : "bf8c40adfc624f0683720d0b6a31283b9962cdaa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94b6fa9e2a018f95b88b5b7ba1146103a94d313",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8996",
       "triggerID" : "e94b6fa9e2a018f95b88b5b7ba1146103a94d313",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6256df86eba2c9a54dc26c8d32c20d6399bfc1f3",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9002",
       "triggerID" : "6256df86eba2c9a54dc26c8d32c20d6399bfc1f3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e94b6fa9e2a018f95b88b5b7ba1146103a94d313 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8996) 
   * 6256df86eba2c9a54dc26c8d32c20d6399bfc1f3 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9002) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol edited a comment on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol edited a comment on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-717170059


   I went through the issues the checker found, but most of them appear to be invalid:
   
   Contradicting statement(similar instances exist for kinesis, flink-oss-fs-hadoop, and possibly others):
   
   ```
   Could not find dependency org.apache.commons:commons-compress:1.20 in NOTICE file /__w/2/s/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE
   Dependency com.apache.commons:commons-compress:1.20 is mentioned in NOTICE file /__w/2/s/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE, but is not expected there
   ```
   
   The flink-dist warnings appear to be mostly incorrect; akka and slf4j are indeed bundled and hence should be listed in the NOTICE. I haven't checked other entries.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] JingsongLi commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517165862



##########
File path: flink-connectors/flink-connector-hive/src/main/resources/META-INF/NOTICE
##########
@@ -7,7 +7,6 @@ The Apache Software Foundation (http://www.apache.org/).
 This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt)
 
 - org.apache.parquet:parquet-hadoop:1.11.1
-- org.apache.parquet:parquet-format:2.4.0

Review comment:
       This is because flink-connector-hive bundle the flink-parquet.
   The parquet version of previous flink-parquet required this `parquet-format:2.4.0`.
   But after https://issues.apache.org/jira/browse/FLINK-19137 , the new parquet version don't need this dependency.
   
   We can just remove it, I think this is a mistake of https://github.com/apache/flink/commit/fb4ebbb1941c37fd1176e31052529f6cdd92faff




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517213870



##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+//CHECKSTYLE.OFF: regexp|imports
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+//CHECKSTYLE.ON: regexp|imports
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), Paths.get(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:[^:]+:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	// Examples:
+	// "- org.apache.htrace:htrace-core:3.1.0-incubating"
+	// or
+	// "This project bundles "net.jcip:jcip-annotations:1.0".
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, Path root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithBundledDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithBundledDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithBundledDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<Path> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithBundledDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (Path noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithBundledDependencies, noticeFile);
+		}
+
+		return severeIssueCount;
+	}
+
+	private static int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<Path> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(Path noticeFile) {
+		Path moduleDirectory = noticeFile.getParent() // META-INF
+			.getParent() // resources
+			.getParent() // main
+			.getParent() // src
+			.getParent(); // <-- module name
+		return moduleDirectory.getFileName().toString();
+	}
+
+	private static int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, Path noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		List<String> noticeContents = Files.readAllLines(noticeFile);
+
+		if (noticeContents.isEmpty()) {
+			LOG.error("Notice file empty {}", noticeFile);
+			severeIssueCount++;
+		}
+
+		// first line must be the module name.
+		if (!noticeContents.get(0).equals(moduleName)) {
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, noticeContents.get(0));
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		for (String line : noticeContents) {
+			Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+			if (noticeDependencyMatcher.find()) {
+				String groupId = noticeDependencyMatcher.group(1);
+				String artifactId = noticeDependencyMatcher.group(2);
+				String version = noticeDependencyMatcher.group(3);
+				if (groupId == null && artifactId == null && version == null) { // "bundles" case
+					groupId = noticeDependencyMatcher.group(5);
+					artifactId = noticeDependencyMatcher.group(6);
+					version = noticeDependencyMatcher.group(7);
+				}
+				IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+				if (!declaredDependencies.add(toAdd)) {
+					LOG.error("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					severeIssueCount++;
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Collection<IncludedDependency> expectedDependencies = modulesWithShadedDependencies.get(moduleName);
+		for (IncludedDependency expectedDependency : expectedDependencies) {
+			if (!declaredDependencies.contains(expectedDependency)) {
+				LOG.error("Could not find dependency {} in NOTICE file {}", expectedDependency, noticeFile);
+				severeIssueCount++;
+			}
+		}
+
+		boolean moduleDefinesExcessDependencies = MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName);
+
+		// print all dependencies defined in NOTICE file, which were not expected
+		for (IncludedDependency declaredDependency : declaredDependencies) {
+			if (!expectedDependencies.contains(declaredDependency)) {
+				if (moduleDefinesExcessDependencies) {
+					LOG.debug(
+						"Dependency {} is mentioned in NOTICE file {}, but was not mentioned by the build output as a bundled dependency",
+						declaredDependency,
+						noticeFile);
+				} else {
+					LOG.warn(
+						"Dependency {} is mentioned in NOTICE file {}, but is not expected there",
+						declaredDependency,
+						noticeFile);
+				}
+
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static List<Path> findNoticeFiles(Path root) throws IOException {
+		return Files.walk(root)
+			.filter(file -> {
+				int nameCount = file.getNameCount();
+				return file.getName(nameCount - 3).toString().equals("resources")
+					&& file.getName(nameCount - 2).toString().equals("META-INF")
+					&& file.getName(nameCount - 1).toString().equals("NOTICE");
+			})
+			.collect(Collectors.toList());
+	}
+
+	private static Multimap<String, IncludedDependency> parseModulesFromBuildResult(File buildResult) throws IOException {
+		Multimap<String, IncludedDependency> result = ArrayListMultimap.create();
+
+		try (Stream<String> lines = Files.lines(buildResult.toPath())) {
+			//String line;
+			String currentShadeModule = null;
+			String currentDependencyCopyModule = null;
+			for (String line : (Iterable<String>) lines::iterator) {

Review comment:
       yeah, but I can not modify the `currentShadeModule` variables then. Of course I could put them into some container, but that would be ugly.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-720939621


   Thanks a lot for your feedback. I pushed a commit addressing your comments.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-717905033


   Let's add them all here.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-721611660


   Thanks a lot for your review. I addressed all outstanding comments.
   Once you declare the changes so far to be good to be merged, I'll rebase to master. We might have additional licensing issues to address then .. let's see 😃 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-716686693


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e58da69e4aeb75fddcdb08c1ae89c2bc9789902a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8322",
       "triggerID" : "e58da69e4aeb75fddcdb08c1ae89c2bc9789902a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54cc6f4d9ff39c498698be95e0640ab9b778b11f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8685",
       "triggerID" : "54cc6f4d9ff39c498698be95e0640ab9b778b11f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be6463deec21201f41db750bf996f98723868b64",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8846",
       "triggerID" : "be6463deec21201f41db750bf996f98723868b64",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bf8c40adfc624f0683720d0b6a31283b9962cdaa",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8982",
       "triggerID" : "bf8c40adfc624f0683720d0b6a31283b9962cdaa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94b6fa9e2a018f95b88b5b7ba1146103a94d313",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8996",
       "triggerID" : "e94b6fa9e2a018f95b88b5b7ba1146103a94d313",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * be6463deec21201f41db750bf996f98723868b64 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8846) 
   * bf8c40adfc624f0683720d0b6a31283b9962cdaa Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8982) 
   * e94b6fa9e2a018f95b88b5b7ba1146103a94d313 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8996) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-718145230


   > There is no benefit in having the generated files in git, it could just be step of the release process
   
   One benefit would be, that our release process is less complicated
   Second benefit is that it is easier to check that the generated NOTICE files are valid when you have to regenerate and commit them. With your approach, we would wait with that until the release, and it would only be done by a few "experts".
   
   I'm undecided here, just discussing to understand our options.
   I'll soon push an update that resolves all errors we have so far in our notice files.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-721649393


   It seems like you missed [this](https://github.com/apache/flink/pull/13796#discussion_r516602034) comment, but we can rebase anyway because I don't have concerns in terms of correctness.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517255590



##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+//CHECKSTYLE.OFF: regexp|imports
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+//CHECKSTYLE.ON: regexp|imports
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), Paths.get(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:[^:]+:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	// Examples:
+	// "- org.apache.htrace:htrace-core:3.1.0-incubating"
+	// or
+	// "This project bundles "net.jcip:jcip-annotations:1.0".
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, Path root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithBundledDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithBundledDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithBundledDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<Path> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithBundledDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (Path noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithBundledDependencies, noticeFile);
+		}
+
+		return severeIssueCount;
+	}
+
+	private static int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<Path> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(Path noticeFile) {
+		Path moduleDirectory = noticeFile.getParent() // META-INF
+			.getParent() // resources
+			.getParent() // main
+			.getParent() // src
+			.getParent(); // <-- module name
+		return moduleDirectory.getFileName().toString();
+	}
+
+	private static int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, Path noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		List<String> noticeContents = Files.readAllLines(noticeFile);
+
+		if (noticeContents.isEmpty()) {
+			LOG.error("Notice file empty {}", noticeFile);
+			severeIssueCount++;
+		}
+
+		// first line must be the module name.
+		if (!noticeContents.get(0).equals(moduleName)) {
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, noticeContents.get(0));
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		for (String line : noticeContents) {
+			Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+			if (noticeDependencyMatcher.find()) {
+				String groupId = noticeDependencyMatcher.group(1);
+				String artifactId = noticeDependencyMatcher.group(2);
+				String version = noticeDependencyMatcher.group(3);
+				if (groupId == null && artifactId == null && version == null) { // "bundles" case
+					groupId = noticeDependencyMatcher.group(5);
+					artifactId = noticeDependencyMatcher.group(6);
+					version = noticeDependencyMatcher.group(7);
+				}
+				IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+				if (!declaredDependencies.add(toAdd)) {
+					LOG.error("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					severeIssueCount++;
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Collection<IncludedDependency> expectedDependencies = modulesWithShadedDependencies.get(moduleName);
+		for (IncludedDependency expectedDependency : expectedDependencies) {
+			if (!declaredDependencies.contains(expectedDependency)) {
+				LOG.error("Could not find dependency {} in NOTICE file {}", expectedDependency, noticeFile);
+				severeIssueCount++;
+			}
+		}
+
+		boolean moduleDefinesExcessDependencies = MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName);
+
+		// print all dependencies defined in NOTICE file, which were not expected
+		for (IncludedDependency declaredDependency : declaredDependencies) {
+			if (!expectedDependencies.contains(declaredDependency)) {
+				if (moduleDefinesExcessDependencies) {
+					LOG.debug(
+						"Dependency {} is mentioned in NOTICE file {}, but was not mentioned by the build output as a bundled dependency",
+						declaredDependency,
+						noticeFile);
+				} else {
+					LOG.warn(
+						"Dependency {} is mentioned in NOTICE file {}, but is not expected there",
+						declaredDependency,
+						noticeFile);
+				}
+
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static List<Path> findNoticeFiles(Path root) throws IOException {
+		return Files.walk(root)
+			.filter(file -> {
+				int nameCount = file.getNameCount();
+				return file.getName(nameCount - 3).toString().equals("resources")
+					&& file.getName(nameCount - 2).toString().equals("META-INF")
+					&& file.getName(nameCount - 1).toString().equals("NOTICE");
+			})
+			.collect(Collectors.toList());
+	}
+
+	private static Multimap<String, IncludedDependency> parseModulesFromBuildResult(File buildResult) throws IOException {
+		Multimap<String, IncludedDependency> result = ArrayListMultimap.create();
+
+		try (Stream<String> lines = Files.lines(buildResult.toPath())) {
+			//String line;
+			String currentShadeModule = null;
+			String currentDependencyCopyModule = null;
+			for (String line : (Iterable<String>) lines::iterator) {

Review comment:
       but let's leave it like it is




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517157132



##########
File path: flink-connectors/flink-connector-hive/src/main/resources/META-INF/NOTICE
##########
@@ -7,7 +7,6 @@ The Apache Software Foundation (http://www.apache.org/).
 This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt)
 
 - org.apache.parquet:parquet-hadoop:1.11.1
-- org.apache.parquet:parquet-format:2.4.0

Review comment:
       @JingsongLi Can you double check that it is fine to remove `org.apache.parquet:parquet-format:2.4.0` from the NOTICE file. @tillrohrmann added it in a hotfix in July: https://github.com/apache/flink/commit/4df425bde344aee8f5a0208df8f5377342af25ff.
   In the current master, the dependency doesn't show up anymore in the maven shade output of the module.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r515952694



##########
File path: flink-connectors/flink-connector-hive/src/main/resources/META-INF/NOTICE
##########
@@ -7,7 +7,6 @@ The Apache Software Foundation (http://www.apache.org/).
 This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt)
 
 - org.apache.parquet:parquet-hadoop:1.11.1
-- org.apache.parquet:parquet-format:2.4.0

Review comment:
       Well, we are deploying all those flink-sql-connector-hive-{1.22,2.2.0,2.3.6,3.1.2} versions. But they have their own NOTICE files.
   The shade plugin output for `flink-connector-hive` doesn't include this dependency. We are not shading `flink-connector-hive` multiple times for different versions.

##########
File path: tools/azure-pipelines/jobs-template.yml
##########
@@ -229,7 +229,7 @@ jobs:
         IT_CASE_S3_BUCKET: $(SECRET_S3_BUCKET)
         IT_CASE_S3_ACCESS_KEY: $(SECRET_S3_ACCESS_KEY)
         IT_CASE_S3_SECRET_KEY: $(SECRET_S3_SECRET_KEY)
-      condition: not(eq(variables['SKIP'], '1'))
+      condition: and(succeeded(),not(eq(variables['SKIP'], '1')))

Review comment:
       I will put it into a separate hotfix.

##########
File path: flink-connectors/flink-sql-connector-hbase-1.4/pom.xml
##########
@@ -87,6 +87,7 @@ under the License.
 									<exclude>org.apache.hbase:hbase-metrics*</exclude>
 									<exclude>org.apache.hbase:hbase-server*</exclude>
 									<exclude>org.apache.hbase:hbase-hadoop*-compat</exclude>
+									<exclude>org.apache.hbase:hbase-common:jar:tests</exclude>

Review comment:
       ```
   [INFO] --- maven-shade-plugin:3.1.1:shade (shade-flink) @ flink-sql-connector-hbase-1.4_2.11 ---
   [INFO] Including org.apache.flink:flink-connector-hbase-1.4_2.11:jar:1.12-SNAPSHOT in the shaded jar.
   [INFO] Including org.apache.flink:flink-connector-hbase-base_2.11:jar:1.12-SNAPSHOT in the shaded jar.
   [INFO] Excluding org.apache.hbase:hbase-server:jar:1.4.3 from the shaded jar.
   [INFO] Including org.apache.hbase:hbase-common:jar:1.4.3 in the shaded jar.
   [INFO] Excluding org.apache.avro:avro:jar:1.10.0 from the shaded jar.
   [INFO] Excluding com.fasterxml.jackson.core:jackson-core:jar:2.10.1 from the shaded jar.
   [INFO] Excluding com.fasterxml.jackson.core:jackson-databind:jar:2.10.1 from the shaded jar.
   [INFO] Excluding com.fasterxml.jackson.core:jackson-annotations:jar:2.10.1 from the shaded jar.
   [INFO] Excluding com.github.stephenc.findbugs:findbugs-annotations:jar:1.3.9-1 from the shaded jar.
   [INFO] Including org.apache.hbase:hbase-protocol:jar:1.4.3 in the shaded jar.
   [INFO] Including org.apache.hbase:hbase-procedure:jar:1.4.3 in the shaded jar.
   [INFO] Including org.apache.hbase:hbase-common:jar:tests:1.4.3 in the shaded jar.
   ```
   
   I believe these are test classes only, which we don't need to ship to our users?

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Scanner;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), new File(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:3.1.1:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, File root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithShadedDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithShadedDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithShadedDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<File> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithShadedDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (File noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithShadedDependencies, noticeFile);
+		}
+
+		// find modules included in flink-dist
+
+		return severeIssueCount;
+	}
+
+	private int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<File> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(File noticeFile) {
+		File moduleFile = noticeFile.getParentFile() // META-INF
+			.getParentFile() // resources
+			.getParentFile() // main
+			.getParentFile() // src
+			.getParentFile(); // <-- module name
+		return moduleFile.getName();
+	}
+
+	private int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, File noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		String noticeContents = readFile(noticeFile.toPath());
+		if (!noticeContents.startsWith(moduleName)) {
+			String firstLine = noticeContents.substring(0, noticeContents.indexOf('\n'));
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, firstLine);
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		try (BufferedReader br = new BufferedReader(new StringReader(noticeContents))) {
+			String line;
+			while ((line = br.readLine()) != null) {
+				Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+				if (noticeDependencyMatcher.find()) {
+					String groupId = noticeDependencyMatcher.group(1);
+					String artifactId = noticeDependencyMatcher.group(2);
+					String version = noticeDependencyMatcher.group(3);
+					if (groupId == null && artifactId == null && version == null) { // "bundles" case
+						groupId = noticeDependencyMatcher.group(5);
+						artifactId = noticeDependencyMatcher.group(6);
+						version = noticeDependencyMatcher.group(7);
+					}
+					IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+					if (!declaredDependencies.add(toAdd)) {
+						LOG.warn("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					}
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Set<IncludedDependency> expectedDependencies = new HashSet<>(modulesWithShadedDependencies.get(moduleName));
+		expectedDependencies.removeAll(declaredDependencies);
+		for (IncludedDependency missingDependency : expectedDependencies) {
+			LOG.error("Could not find dependency {} in NOTICE file {}", missingDependency, noticeFile);
+			severeIssueCount++;
+		}
+
+		if (!MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName)) {
+			// print all dependencies defined in NOTICE file, which were not expected
+			Set<IncludedDependency> excessDependencies = new HashSet<>(declaredDependencies);
+			excessDependencies.removeAll(modulesWithShadedDependencies.get(moduleName));
+			for (IncludedDependency excessDependency : excessDependencies) {
+				LOG.warn("Dependency {} is mentioned in NOTICE file {}, but is not expected there", excessDependency, noticeFile);
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static String readFile(Path path) throws IOException {
+		byte[] encoded = Files.readAllBytes(path);
+		return new String(encoded, Charset.defaultCharset());
+	}
+
+	private List<File> findNoticeFiles(File root) throws IOException {
+		return Files.walk(root.toPath())
+			.filter(file -> {
+				int nameCount = file.getNameCount();
+				return file.getName(nameCount - 1).toString().equals("NOTICE")
+					&& file.getName(nameCount - 2).toString().equals("META-INF")
+					&& file.getName(nameCount - 3).toString().equals("resources");
+			})
+			.map(Path::toFile)
+			.collect(Collectors.toList());
+	}
+
+	private Multimap<String, IncludedDependency> parseModulesFromBuildResult(File buildResult) throws IOException {
+		Multimap<String, IncludedDependency> result = ArrayListMultimap.create();
+		try (BufferedReader br = new BufferedReader(new FileReader(buildResult))) {

Review comment:
       I agree that this is an alternative approach. It would imply materializing the entire build output as strings into the JVM. I'm not sure if the code is a lot more readable when having multiple nested loops fiddling with the line indexes.
   For the sake of my time, I would like to keep the code as-is for now. I agree it is not perfectly easy to understand, but the improvement wouldn't be substantial enough to pay the performance and reimplementation price.

##########
File path: flink-connectors/flink-sql-connector-hbase-1.4/pom.xml
##########
@@ -87,6 +87,7 @@ under the License.
 									<exclude>org.apache.hbase:hbase-metrics*</exclude>
 									<exclude>org.apache.hbase:hbase-server*</exclude>
 									<exclude>org.apache.hbase:hbase-hadoop*-compat</exclude>
+									<exclude>org.apache.hbase:hbase-common:jar:tests</exclude>

Review comment:
       Yes, true. I will look into this.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-717906443


   Having such a mapping was my idea as well; the explicit failure if a dependency was not found also serves as a reminder to check the license for compatibility.
   I don't think this would be tooo difficult tbh.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-717761562


   Thanks a lot for taking a look. I believe the statement by the tool is correct: The notice file contains the following dependency: `com.apache.commons:commons-compress:1.20`, but the expected dependency is `org.apache.commons:commons-compress:1.20`. Notice the `com.` vs `org.`
   
   Do you agree to how the tool is integrated into the build process in priciple? If so, I will check and fix all dependency issues found by the tool, so that we can merge it.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-718130365


   There is no benefit in having the generated files in git, it could just be step of the release process.
   
   On CI we would compile Flink and pipe the build output into a file, then run the generator with this file as input to ensure that the NOTICE files can be generated.
   
   > I've quickly looked at building our own shade resource transformer
   
   FYI: Last time I tried it wasn't possible to write custom transformers, and even if you could it would interfere with other transformers touching the notice file (like the appender).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517179535



##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -120,87 +122,100 @@ private int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modul
 		return severeIssueCount;
 	}
 
-	private static String getModuleFromNoticeFile(File noticeFile) {
-		File moduleFile = noticeFile.getParentFile() // META-INF
-			.getParentFile() // resources
-			.getParentFile() // main
-			.getParentFile() // src
-			.getParentFile(); // <-- module name
-		return moduleFile.getName();
+	private static String getModuleFromNoticeFile(Path noticeFile) {
+		Path moduleDirectory = noticeFile.getParent() // META-INF
+			.getParent() // resources
+			.getParent() // main
+			.getParent() // src
+			.getParent(); // <-- module name
+		return moduleDirectory.getFileName().toString();
 	}
 
-	private int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, File noticeFile) throws IOException {
+	private static int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, Path noticeFile) throws IOException {
 		int severeIssueCount = 0;
 		String moduleName = getModuleFromNoticeFile(noticeFile);
 
 		// 1st line contains module name
-		String noticeContents = readFile(noticeFile.toPath());
-		if (!noticeContents.startsWith(moduleName)) {
-			String firstLine = noticeContents.substring(0, noticeContents.indexOf('\n'));
-			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, firstLine);
+		List<String> noticeContents = Files.readAllLines(noticeFile);
+
+		if (noticeContents.isEmpty()) {
+			LOG.error("Notice file empty {}", noticeFile);
+			severeIssueCount++;
+		}
+
+		// first line must be the module name.
+		if (!noticeContents.get(0).equals(moduleName)) {
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, noticeContents.get(0));
 		}
 
 		// collect all declared dependencies from NOTICE file
 		Set<IncludedDependency> declaredDependencies = new HashSet<>();
-		try (BufferedReader br = new BufferedReader(new StringReader(noticeContents))) {
-			String line;
-			while ((line = br.readLine()) != null) {
-				Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
-				if (noticeDependencyMatcher.find()) {
-					String groupId = noticeDependencyMatcher.group(1);
-					String artifactId = noticeDependencyMatcher.group(2);
-					String version = noticeDependencyMatcher.group(3);
-					if (groupId == null && artifactId == null && version == null) { // "bundles" case
-						groupId = noticeDependencyMatcher.group(5);
-						artifactId = noticeDependencyMatcher.group(6);
-						version = noticeDependencyMatcher.group(7);
-					}
-					IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
-					if (!declaredDependencies.add(toAdd)) {
-						LOG.warn("Dependency {} has been declared twice in module {}", toAdd, moduleName);
-					}
+		for (String line : noticeContents) {
+			Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+			if (noticeDependencyMatcher.find()) {
+				String groupId = noticeDependencyMatcher.group(1);
+				String artifactId = noticeDependencyMatcher.group(2);
+				String version = noticeDependencyMatcher.group(3);
+				if (groupId == null && artifactId == null && version == null) { // "bundles" case
+					groupId = noticeDependencyMatcher.group(5);
+					artifactId = noticeDependencyMatcher.group(6);
+					version = noticeDependencyMatcher.group(7);
+				}
+				IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+				if (!declaredDependencies.add(toAdd)) {
+					LOG.error("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					severeIssueCount++;
 				}
 			}
 		}
 		// print all dependencies missing from NOTICE file
-		Set<IncludedDependency> expectedDependencies = new HashSet<>(modulesWithShadedDependencies.get(moduleName));
-		expectedDependencies.removeAll(declaredDependencies);
-		for (IncludedDependency missingDependency : expectedDependencies) {
-			LOG.error("Could not find dependency {} in NOTICE file {}", missingDependency, noticeFile);
-			severeIssueCount++;
+		Collection<IncludedDependency> expectedDependencies = modulesWithShadedDependencies.get(moduleName);
+		for (IncludedDependency expectedDependency : expectedDependencies) {
+			if (!declaredDependencies.contains(expectedDependency)) {
+				LOG.error("Could not find dependency {} in NOTICE file {}", expectedDependency, noticeFile);
+				severeIssueCount++;
+			}
 		}
 
-		if (!MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName)) {
-			// print all dependencies defined in NOTICE file, which were not expected
-			Set<IncludedDependency> excessDependencies = new HashSet<>(declaredDependencies);
-			excessDependencies.removeAll(modulesWithShadedDependencies.get(moduleName));
-			for (IncludedDependency excessDependency : excessDependencies) {
-				LOG.warn("Dependency {} is mentioned in NOTICE file {}, but is not expected there", excessDependency, noticeFile);
+		boolean moduleDefinesExcessDependencies = MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName);
+
+		// print all dependencies defined in NOTICE file, which were not expected
+		Set<IncludedDependency> excessDependencies = new HashSet<>(declaredDependencies);

Review comment:
       🤦 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol edited a comment on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol edited a comment on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-721707763


   This is what I see in the dependencies section, and I don't see a compile scope.
   
   ```
   	<dependencies>
   		<dependency>
   			<groupId>com.google.guava</groupId>
   			<artifactId>guava</artifactId>
   			<version>30.0-jre</version>
   		</dependency>
   		<dependency>
   			<groupId>org.apache.logging.log4j</groupId>
   			<artifactId>log4j-slf4j-impl</artifactId>
   		</dependency>
   		<dependency>
   			<groupId>org.apache.logging.log4j</groupId>
   			<artifactId>log4j-api</artifactId>
   		</dependency>
   		<dependency>
   			<groupId>org.apache.logging.log4j</groupId>
   			<artifactId>log4j-core</artifactId>
   		</dependency>
   	</dependencies>
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-721712709


   I rebased the change to master, locally the license check has passed.
   
   Once you give approval, I'll clean up the history, rebase & run it on CI one more time and then merge it.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-721747048


   We have explicit test dependencies on log4j in the root pom. They are not defined in the dependency management, so that not every module has to declare this stuff again.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517301192



##########
File path: flink-connectors/flink-sql-connector-hbase-1.4/pom.xml
##########
@@ -87,6 +87,7 @@ under the License.
 									<exclude>org.apache.hbase:hbase-metrics*</exclude>
 									<exclude>org.apache.hbase:hbase-server*</exclude>
 									<exclude>org.apache.hbase:hbase-hadoop*-compat</exclude>
+									<exclude>org.apache.hbase:hbase-common:jar:tests</exclude>

Review comment:
       I'll try the test dependency approach, and I've filed a ticket: https://issues.apache.org/jira/browse/FLINK-19971




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-716686693


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e58da69e4aeb75fddcdb08c1ae89c2bc9789902a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8322",
       "triggerID" : "e58da69e4aeb75fddcdb08c1ae89c2bc9789902a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54cc6f4d9ff39c498698be95e0640ab9b778b11f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8685",
       "triggerID" : "54cc6f4d9ff39c498698be95e0640ab9b778b11f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be6463deec21201f41db750bf996f98723868b64",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8846",
       "triggerID" : "be6463deec21201f41db750bf996f98723868b64",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bf8c40adfc624f0683720d0b6a31283b9962cdaa",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8982",
       "triggerID" : "bf8c40adfc624f0683720d0b6a31283b9962cdaa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94b6fa9e2a018f95b88b5b7ba1146103a94d313",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e94b6fa9e2a018f95b88b5b7ba1146103a94d313",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * be6463deec21201f41db750bf996f98723868b64 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8846) 
   * bf8c40adfc624f0683720d0b6a31283b9962cdaa Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8982) 
   * e94b6fa9e2a018f95b88b5b7ba1146103a94d313 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-721707763


   This is what I see in the dependencies section, and I don't see a compile scope.
   
   ```
   <dependencies>
   		<dependency>
   			<groupId>com.google.guava</groupId>
   			<artifactId>guava</artifactId>
   			<version>30.0-jre</version>
   		</dependency>
   		<dependency>
   			<groupId>org.apache.logging.log4j</groupId>
   			<artifactId>log4j-slf4j-impl</artifactId>
   		</dependency>
   		<dependency>
   			<groupId>org.apache.logging.log4j</groupId>
   			<artifactId>log4j-api</artifactId>
   		</dependency>
   		<dependency>
   			<groupId>org.apache.logging.log4j</groupId>
   			<artifactId>log4j-core</artifactId>
   		</dependency>
   	</dependencies>
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-716686693


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e58da69e4aeb75fddcdb08c1ae89c2bc9789902a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8322",
       "triggerID" : "e58da69e4aeb75fddcdb08c1ae89c2bc9789902a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54cc6f4d9ff39c498698be95e0640ab9b778b11f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8685",
       "triggerID" : "54cc6f4d9ff39c498698be95e0640ab9b778b11f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be6463deec21201f41db750bf996f98723868b64",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8846",
       "triggerID" : "be6463deec21201f41db750bf996f98723868b64",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bf8c40adfc624f0683720d0b6a31283b9962cdaa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bf8c40adfc624f0683720d0b6a31283b9962cdaa",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * be6463deec21201f41db750bf996f98723868b64 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8846) 
   * bf8c40adfc624f0683720d0b6a31283b9962cdaa UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517211405



##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+//CHECKSTYLE.OFF: regexp|imports
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+//CHECKSTYLE.ON: regexp|imports
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), Paths.get(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:[^:]+:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	// Examples:
+	// "- org.apache.htrace:htrace-core:3.1.0-incubating"
+	// or
+	// "This project bundles "net.jcip:jcip-annotations:1.0".
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, Path root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithBundledDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithBundledDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithBundledDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<Path> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithBundledDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (Path noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithBundledDependencies, noticeFile);
+		}
+
+		return severeIssueCount;
+	}
+
+	private static int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<Path> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(Path noticeFile) {
+		Path moduleDirectory = noticeFile.getParent() // META-INF
+			.getParent() // resources
+			.getParent() // main
+			.getParent() // src
+			.getParent(); // <-- module name
+		return moduleDirectory.getFileName().toString();
+	}
+
+	private static int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, Path noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		List<String> noticeContents = Files.readAllLines(noticeFile);
+
+		if (noticeContents.isEmpty()) {
+			LOG.error("Notice file empty {}", noticeFile);
+			severeIssueCount++;
+		}
+
+		// first line must be the module name.
+		if (!noticeContents.get(0).equals(moduleName)) {
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, noticeContents.get(0));
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		for (String line : noticeContents) {
+			Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+			if (noticeDependencyMatcher.find()) {
+				String groupId = noticeDependencyMatcher.group(1);
+				String artifactId = noticeDependencyMatcher.group(2);
+				String version = noticeDependencyMatcher.group(3);
+				if (groupId == null && artifactId == null && version == null) { // "bundles" case
+					groupId = noticeDependencyMatcher.group(5);
+					artifactId = noticeDependencyMatcher.group(6);
+					version = noticeDependencyMatcher.group(7);
+				}
+				IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+				if (!declaredDependencies.add(toAdd)) {
+					LOG.error("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					severeIssueCount++;
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Collection<IncludedDependency> expectedDependencies = modulesWithShadedDependencies.get(moduleName);
+		for (IncludedDependency expectedDependency : expectedDependencies) {
+			if (!declaredDependencies.contains(expectedDependency)) {
+				LOG.error("Could not find dependency {} in NOTICE file {}", expectedDependency, noticeFile);
+				severeIssueCount++;
+			}
+		}
+
+		boolean moduleDefinesExcessDependencies = MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName);
+
+		// print all dependencies defined in NOTICE file, which were not expected
+		for (IncludedDependency declaredDependency : declaredDependencies) {
+			if (!expectedDependencies.contains(declaredDependency)) {
+				if (moduleDefinesExcessDependencies) {
+					LOG.debug(
+						"Dependency {} is mentioned in NOTICE file {}, but was not mentioned by the build output as a bundled dependency",
+						declaredDependency,
+						noticeFile);
+				} else {
+					LOG.warn(
+						"Dependency {} is mentioned in NOTICE file {}, but is not expected there",
+						declaredDependency,
+						noticeFile);
+				}
+
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static List<Path> findNoticeFiles(Path root) throws IOException {
+		return Files.walk(root)
+			.filter(file -> {
+				int nameCount = file.getNameCount();
+				return file.getName(nameCount - 3).toString().equals("resources")
+					&& file.getName(nameCount - 2).toString().equals("META-INF")
+					&& file.getName(nameCount - 1).toString().equals("NOTICE");
+			})
+			.collect(Collectors.toList());
+	}
+
+	private static Multimap<String, IncludedDependency> parseModulesFromBuildResult(File buildResult) throws IOException {
+		Multimap<String, IncludedDependency> result = ArrayListMultimap.create();
+
+		try (Stream<String> lines = Files.lines(buildResult.toPath())) {
+			//String line;
+			String currentShadeModule = null;
+			String currentDependencyCopyModule = null;
+			for (String line : (Iterable<String>) lines::iterator) {
+				Matcher nextShadeModuleMatcher = SHADE_NEXT_MODULE_PATTERN.matcher(line);
+				if (nextShadeModuleMatcher.find()) {
+					currentShadeModule = nextShadeModuleMatcher.group(2);
+				}
+
+				Matcher nextDependencyCopyModuleMatcher = DEPENDENCY_COPY_NEXT_MODULE_PATTERN.matcher(line);
+				if (nextDependencyCopyModuleMatcher.find()) {
+					currentDependencyCopyModule = nextDependencyCopyModuleMatcher.group(1);
+				}
+
+				if (currentShadeModule != null) {
+					Matcher includeMatcher = SHADE_INCLUDE_MODULE_PATTERN.matcher(line);
+					if (includeMatcher.find()) {
+						String groupId = includeMatcher.group(1);
+						String artifactId = includeMatcher.group(2);
+						String version = includeMatcher.group(3);
+						if (!"org.apache.flink".equals(groupId)) {
+							result.put(currentShadeModule, IncludedDependency.create(groupId, artifactId, version));
+						}
+					}
+				}
+
+				if (currentDependencyCopyModule != null) {
+					Matcher copyMatcher = DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN.matcher(line);
+					if (copyMatcher.find()) {
+						String groupId = copyMatcher.group(1);
+						String artifactId = copyMatcher.group(2);
+						String version = copyMatcher.group(3);
+						if (!"org.apache.flink".equals(groupId)) {
+							result.put(currentDependencyCopyModule, IncludedDependency.create(groupId, artifactId, version));
+						}
+					}
+				}
+				if (line.contains("Replacing original artifact with shaded artifact")) {
+					currentShadeModule = null;
+				}
+				if (line.contains("Copying")) {
+					currentDependencyCopyModule = null;
+				}
+			}

Review comment:
       let's do it like this then:
   
   ```suggestion
   			String currentShadeModule = null;
   			String currentDependencyCopyModule = null;
   			lines.forEach(line -> {
   				Matcher nextShadeModuleMatcher = SHADE_NEXT_MODULE_PATTERN.matcher(line);
   				if (nextShadeModuleMatcher.find()) {
   					currentShadeModule = nextShadeModuleMatcher.group(2);
   				}
   
   				Matcher nextDependencyCopyModuleMatcher = DEPENDENCY_COPY_NEXT_MODULE_PATTERN.matcher(line);
   				if (nextDependencyCopyModuleMatcher.find()) {
   					currentDependencyCopyModule = nextDependencyCopyModuleMatcher.group(1);
   				}
   
   				if (currentShadeModule != null) {
   					Matcher includeMatcher = SHADE_INCLUDE_MODULE_PATTERN.matcher(line);
   					if (includeMatcher.find()) {
   						String groupId = includeMatcher.group(1);
   						String artifactId = includeMatcher.group(2);
   						String version = includeMatcher.group(3);
   						if (!"org.apache.flink".equals(groupId)) {
   							result.put(currentShadeModule, IncludedDependency.create(groupId, artifactId, version));
   						}
   					}
   				}
   
   				if (currentDependencyCopyModule != null) {
   					Matcher copyMatcher = DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN.matcher(line);
   					if (copyMatcher.find()) {
   						String groupId = copyMatcher.group(1);
   						String artifactId = copyMatcher.group(2);
   						String version = copyMatcher.group(3);
   						if (!"org.apache.flink".equals(groupId)) {
   							result.put(currentDependencyCopyModule, IncludedDependency.create(groupId, artifactId, version));
   						}
   					}
   				}
   				if (line.contains("Replacing original artifact with shaded artifact")) {
   					currentShadeModule = null;
   				}
   				if (line.contains("Copying")) {
   					currentDependencyCopyModule = null;
   				}
   			});
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-721747784


   This is also why flink-dist has explicit compile depencies on log4j ;)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517172264



##########
File path: flink-connectors/flink-connector-hive/src/main/resources/META-INF/NOTICE
##########
@@ -7,7 +7,6 @@ The Apache Software Foundation (http://www.apache.org/).
 This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt)
 
 - org.apache.parquet:parquet-hadoop:1.11.1
-- org.apache.parquet:parquet-format:2.4.0

Review comment:
       Thanks a lot for confirming. I just searched the entire Flink build output, and `parquet-format` is not mentioned there at all.
   I'll remove this entry from the NOTICE file then.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517211912



##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+//CHECKSTYLE.OFF: regexp|imports
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+//CHECKSTYLE.ON: regexp|imports
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), Paths.get(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:[^:]+:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	// Examples:
+	// "- org.apache.htrace:htrace-core:3.1.0-incubating"
+	// or
+	// "This project bundles "net.jcip:jcip-annotations:1.0".
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, Path root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithBundledDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithBundledDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithBundledDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<Path> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithBundledDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (Path noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithBundledDependencies, noticeFile);
+		}
+
+		return severeIssueCount;
+	}
+
+	private static int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<Path> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(Path noticeFile) {
+		Path moduleDirectory = noticeFile.getParent() // META-INF
+			.getParent() // resources
+			.getParent() // main
+			.getParent() // src
+			.getParent(); // <-- module name
+		return moduleDirectory.getFileName().toString();
+	}
+
+	private static int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, Path noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		List<String> noticeContents = Files.readAllLines(noticeFile);
+
+		if (noticeContents.isEmpty()) {
+			LOG.error("Notice file empty {}", noticeFile);
+			severeIssueCount++;
+		}
+
+		// first line must be the module name.
+		if (!noticeContents.get(0).equals(moduleName)) {
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, noticeContents.get(0));
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		for (String line : noticeContents) {
+			Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+			if (noticeDependencyMatcher.find()) {
+				String groupId = noticeDependencyMatcher.group(1);
+				String artifactId = noticeDependencyMatcher.group(2);
+				String version = noticeDependencyMatcher.group(3);
+				if (groupId == null && artifactId == null && version == null) { // "bundles" case
+					groupId = noticeDependencyMatcher.group(5);
+					artifactId = noticeDependencyMatcher.group(6);
+					version = noticeDependencyMatcher.group(7);
+				}
+				IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+				if (!declaredDependencies.add(toAdd)) {
+					LOG.error("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					severeIssueCount++;
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Collection<IncludedDependency> expectedDependencies = modulesWithShadedDependencies.get(moduleName);
+		for (IncludedDependency expectedDependency : expectedDependencies) {
+			if (!declaredDependencies.contains(expectedDependency)) {
+				LOG.error("Could not find dependency {} in NOTICE file {}", expectedDependency, noticeFile);
+				severeIssueCount++;
+			}
+		}
+
+		boolean moduleDefinesExcessDependencies = MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName);
+
+		// print all dependencies defined in NOTICE file, which were not expected
+		for (IncludedDependency declaredDependency : declaredDependencies) {
+			if (!expectedDependencies.contains(declaredDependency)) {
+				if (moduleDefinesExcessDependencies) {
+					LOG.debug(
+						"Dependency {} is mentioned in NOTICE file {}, but was not mentioned by the build output as a bundled dependency",
+						declaredDependency,
+						noticeFile);
+				} else {
+					LOG.warn(
+						"Dependency {} is mentioned in NOTICE file {}, but is not expected there",
+						declaredDependency,
+						noticeFile);
+				}
+
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static List<Path> findNoticeFiles(Path root) throws IOException {
+		return Files.walk(root)
+			.filter(file -> {
+				int nameCount = file.getNameCount();
+				return file.getName(nameCount - 3).toString().equals("resources")
+					&& file.getName(nameCount - 2).toString().equals("META-INF")
+					&& file.getName(nameCount - 1).toString().equals("NOTICE");
+			})
+			.collect(Collectors.toList());
+	}
+
+	private static Multimap<String, IncludedDependency> parseModulesFromBuildResult(File buildResult) throws IOException {
+		Multimap<String, IncludedDependency> result = ArrayListMultimap.create();
+
+		try (Stream<String> lines = Files.lines(buildResult.toPath())) {
+			//String line;
+			String currentShadeModule = null;
+			String currentDependencyCopyModule = null;
+			for (String line : (Iterable<String>) lines::iterator) {
+				Matcher nextShadeModuleMatcher = SHADE_NEXT_MODULE_PATTERN.matcher(line);
+				if (nextShadeModuleMatcher.find()) {
+					currentShadeModule = nextShadeModuleMatcher.group(2);
+				}
+
+				Matcher nextDependencyCopyModuleMatcher = DEPENDENCY_COPY_NEXT_MODULE_PATTERN.matcher(line);
+				if (nextDependencyCopyModuleMatcher.find()) {
+					currentDependencyCopyModule = nextDependencyCopyModuleMatcher.group(1);
+				}
+
+				if (currentShadeModule != null) {
+					Matcher includeMatcher = SHADE_INCLUDE_MODULE_PATTERN.matcher(line);
+					if (includeMatcher.find()) {
+						String groupId = includeMatcher.group(1);
+						String artifactId = includeMatcher.group(2);
+						String version = includeMatcher.group(3);
+						if (!"org.apache.flink".equals(groupId)) {
+							result.put(currentShadeModule, IncludedDependency.create(groupId, artifactId, version));
+						}
+					}
+				}
+
+				if (currentDependencyCopyModule != null) {
+					Matcher copyMatcher = DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN.matcher(line);
+					if (copyMatcher.find()) {
+						String groupId = copyMatcher.group(1);
+						String artifactId = copyMatcher.group(2);
+						String version = copyMatcher.group(3);
+						if (!"org.apache.flink".equals(groupId)) {
+							result.put(currentDependencyCopyModule, IncludedDependency.create(groupId, artifactId, version));
+						}
+					}
+				}
+				if (line.contains("Replacing original artifact with shaded artifact")) {
+					currentShadeModule = null;
+				}
+				if (line.contains("Copying")) {
+					currentDependencyCopyModule = null;
+				}
+			}

Review comment:
       continuation of https://github.com/apache/flink/pull/13796/files?file-filters%5B%5D=.activation&file-filters%5B%5D=.java&file-filters%5B%5D=.modulelist&file-filters%5B%5D=.properties&file-filters%5B%5D=.sh&file-filters%5B%5D=.yml&file-filters%5B%5D=No+extension#r517211857:
   ```suggestion
   			});
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-716686693


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e58da69e4aeb75fddcdb08c1ae89c2bc9789902a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8322",
       "triggerID" : "e58da69e4aeb75fddcdb08c1ae89c2bc9789902a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54cc6f4d9ff39c498698be95e0640ab9b778b11f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8685",
       "triggerID" : "54cc6f4d9ff39c498698be95e0640ab9b778b11f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be6463deec21201f41db750bf996f98723868b64",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8846",
       "triggerID" : "be6463deec21201f41db750bf996f98723868b64",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bf8c40adfc624f0683720d0b6a31283b9962cdaa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8982",
       "triggerID" : "bf8c40adfc624f0683720d0b6a31283b9962cdaa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94b6fa9e2a018f95b88b5b7ba1146103a94d313",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8996",
       "triggerID" : "e94b6fa9e2a018f95b88b5b7ba1146103a94d313",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6256df86eba2c9a54dc26c8d32c20d6399bfc1f3",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9002",
       "triggerID" : "6256df86eba2c9a54dc26c8d32c20d6399bfc1f3",
       "triggerType" : "PUSH"
     }, {
       "hash" : "501413667977d1d9180cc9297d3549f59bc57f5d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "501413667977d1d9180cc9297d3549f59bc57f5d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 6256df86eba2c9a54dc26c8d32c20d6399bfc1f3 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=9002) 
   * 501413667977d1d9180cc9297d3549f59bc57f5d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-716686693


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e58da69e4aeb75fddcdb08c1ae89c2bc9789902a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8322",
       "triggerID" : "e58da69e4aeb75fddcdb08c1ae89c2bc9789902a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "54cc6f4d9ff39c498698be95e0640ab9b778b11f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8685",
       "triggerID" : "54cc6f4d9ff39c498698be95e0640ab9b778b11f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "be6463deec21201f41db750bf996f98723868b64",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8846",
       "triggerID" : "be6463deec21201f41db750bf996f98723868b64",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bf8c40adfc624f0683720d0b6a31283b9962cdaa",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8982",
       "triggerID" : "bf8c40adfc624f0683720d0b6a31283b9962cdaa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e94b6fa9e2a018f95b88b5b7ba1146103a94d313",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8996",
       "triggerID" : "e94b6fa9e2a018f95b88b5b7ba1146103a94d313",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6256df86eba2c9a54dc26c8d32c20d6399bfc1f3",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6256df86eba2c9a54dc26c8d32c20d6399bfc1f3",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * bf8c40adfc624f0683720d0b6a31283b9962cdaa Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8982) 
   * e94b6fa9e2a018f95b88b5b7ba1146103a94d313 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8996) 
   * 6256df86eba2c9a54dc26c8d32c20d6399bfc1f3 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol edited a comment on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol edited a comment on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-718130365


   There is no benefit in having the generated files in git, it could just be step of the release process. It will imply a second packaging step. (compile + save output -> generate NOTICE -> repackage)
   
   On CI we would compile Flink and pipe the build output into a file, then run the generator with this file as input to ensure that the NOTICE files can be generated.
   
   > I've quickly looked at building our own shade resource transformer
   
   FYI: Last time I tried it wasn't possible to write custom transformers (it _looks_ like you can, but I always had classpath issues), and even if you could it would interfere with other transformers touching the notice file (like the appender), because a file can only be touched by at most 1 transformer.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] JingsongLi commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517165862



##########
File path: flink-connectors/flink-connector-hive/src/main/resources/META-INF/NOTICE
##########
@@ -7,7 +7,6 @@ The Apache Software Foundation (http://www.apache.org/).
 This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt)
 
 - org.apache.parquet:parquet-hadoop:1.11.1
-- org.apache.parquet:parquet-format:2.4.0

Review comment:
       This is because hive bundle the flink-parquet.
   The parquet version of previous flink-parquet required this `parquet-format:2.4.0`.
   But after https://issues.apache.org/jira/browse/FLINK-19137 , the new parquet version don't need this dependency.
   
   We can just remove it, I think this is a mistake of https://github.com/apache/flink/commit/fb4ebbb1941c37fd1176e31052529f6cdd92faff




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r515845233



##########
File path: tools/ci/java-ci-tools/pom.xml
##########
@@ -0,0 +1,84 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+		 xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>org.apache</groupId>
+		<artifactId>apache</artifactId>
+		<version>18</version>
+	</parent>
+
+	<groupId>org.apache.flink</groupId>
+	<artifactId>java-ci-tools</artifactId>
+	<version>1.12-SNAPSHOT</version>
+	<name>Flink : Tools : CI : Java</name>
+
+	<packaging>jar</packaging>
+	<properties>
+		<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+		<maven.compiler.source>1.8</maven.compiler.source>
+		<maven.compiler.target>1.8</maven.compiler.target>

Review comment:
       we could ommit a few of these if the parent pom were the flink-parent pom

##########
File path: tools/ci/java-ci-tools/src/main/resources/modules-skipping-deployment.modulelist
##########
@@ -0,0 +1,43 @@
+################################################################################
+#  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.
+################################################################################
+
+# These modules are not deployed to maven central, despite their use of the shade plugin.
+
+flink-examples-streaming-twitter
+flink-examples-streaming-state-machine
+flink-examples-streaming-gcp-pubsub
+flink-s3-fs-base
+flink-yarn-tests
+flink-docs
+flink-datastream-allround-test
+flink-queryable-state-test
+flink-confluent-schema-registry
+flink-stream-stateful-job-upgrade-test
+flink-elasticsearch7-test
+flink-stream-state-ttl-test
+flink-state-evolution-test
+flink-elasticsearch6-test
+flink-rocksdb-state-memory-control-test
+flink-python-test
+flink-streaming-kinesis-test
+flink-tpch-test
+flink-scala-shell

Review comment:
       this is still included in the distribution, so the notice file must be checked.

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Scanner;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), new File(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:3.1.1:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, File root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithShadedDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithShadedDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithShadedDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<File> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithShadedDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (File noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithShadedDependencies, noticeFile);
+		}
+
+		// find modules included in flink-dist
+
+		return severeIssueCount;
+	}
+
+	private int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<File> noticeFiles) {

Review comment:
       ```suggestion
   	private static int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<File> noticeFiles) {
   ```

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Scanner;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), new File(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:3.1.1:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, File root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithShadedDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithShadedDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithShadedDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<File> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithShadedDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (File noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithShadedDependencies, noticeFile);
+		}
+
+		// find modules included in flink-dist
+
+		return severeIssueCount;
+	}
+
+	private int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<File> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(File noticeFile) {
+		File moduleFile = noticeFile.getParentFile() // META-INF
+			.getParentFile() // resources
+			.getParentFile() // main
+			.getParentFile() // src
+			.getParentFile(); // <-- module name
+		return moduleFile.getName();
+	}
+
+	private int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, File noticeFile) throws IOException {

Review comment:
       ```suggestion
   	private int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, File noticeFile) throws IOException {
   ```suggestion
   	private static int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, File noticeFile) throws IOException {
   ```

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Scanner;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), new File(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:3.1.1:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, File root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithShadedDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithShadedDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithShadedDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<File> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithShadedDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (File noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithShadedDependencies, noticeFile);
+		}
+
+		// find modules included in flink-dist
+
+		return severeIssueCount;
+	}
+
+	private int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<File> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(File noticeFile) {
+		File moduleFile = noticeFile.getParentFile() // META-INF
+			.getParentFile() // resources
+			.getParentFile() // main
+			.getParentFile() // src
+			.getParentFile(); // <-- module name
+		return moduleFile.getName();
+	}
+
+	private int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, File noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		String noticeContents = readFile(noticeFile.toPath());
+		if (!noticeContents.startsWith(moduleName)) {
+			String firstLine = noticeContents.substring(0, noticeContents.indexOf('\n'));
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, firstLine);
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		try (BufferedReader br = new BufferedReader(new StringReader(noticeContents))) {
+			String line;
+			while ((line = br.readLine()) != null) {
+				Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+				if (noticeDependencyMatcher.find()) {
+					String groupId = noticeDependencyMatcher.group(1);
+					String artifactId = noticeDependencyMatcher.group(2);
+					String version = noticeDependencyMatcher.group(3);
+					if (groupId == null && artifactId == null && version == null) { // "bundles" case
+						groupId = noticeDependencyMatcher.group(5);
+						artifactId = noticeDependencyMatcher.group(6);
+						version = noticeDependencyMatcher.group(7);
+					}
+					IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+					if (!declaredDependencies.add(toAdd)) {

Review comment:
       this should be considered a severe issue, because it could also happen if 2 artifacts with the same groupId/artifactId/version but different types are bundled.
   If this ever happens, we want this to fail _somewhere_.

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Scanner;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), new File(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:3.1.1:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, File root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithShadedDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithShadedDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithShadedDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<File> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithShadedDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (File noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithShadedDependencies, noticeFile);
+		}
+
+		// find modules included in flink-dist
+
+		return severeIssueCount;
+	}
+
+	private int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<File> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(File noticeFile) {
+		File moduleFile = noticeFile.getParentFile() // META-INF
+			.getParentFile() // resources
+			.getParentFile() // main
+			.getParentFile() // src
+			.getParentFile(); // <-- module name
+		return moduleFile.getName();
+	}
+
+	private int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, File noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		String noticeContents = readFile(noticeFile.toPath());
+		if (!noticeContents.startsWith(moduleName)) {
+			String firstLine = noticeContents.substring(0, noticeContents.indexOf('\n'));
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, firstLine);
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		try (BufferedReader br = new BufferedReader(new StringReader(noticeContents))) {
+			String line;
+			while ((line = br.readLine()) != null) {
+				Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+				if (noticeDependencyMatcher.find()) {
+					String groupId = noticeDependencyMatcher.group(1);
+					String artifactId = noticeDependencyMatcher.group(2);
+					String version = noticeDependencyMatcher.group(3);
+					if (groupId == null && artifactId == null && version == null) { // "bundles" case
+						groupId = noticeDependencyMatcher.group(5);
+						artifactId = noticeDependencyMatcher.group(6);
+						version = noticeDependencyMatcher.group(7);
+					}
+					IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+					if (!declaredDependencies.add(toAdd)) {
+						LOG.warn("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					}
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Set<IncludedDependency> expectedDependencies = new HashSet<>(modulesWithShadedDependencies.get(moduleName));
+		expectedDependencies.removeAll(declaredDependencies);
+		for (IncludedDependency missingDependency : expectedDependencies) {
+			LOG.error("Could not find dependency {} in NOTICE file {}", missingDependency, noticeFile);
+			severeIssueCount++;
+		}
+
+		if (!MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName)) {
+			// print all dependencies defined in NOTICE file, which were not expected
+			Set<IncludedDependency> excessDependencies = new HashSet<>(declaredDependencies);
+			excessDependencies.removeAll(modulesWithShadedDependencies.get(moduleName));
+			for (IncludedDependency excessDependency : excessDependencies) {
+				LOG.warn("Dependency {} is mentioned in NOTICE file {}, but is not expected there", excessDependency, noticeFile);
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static String readFile(Path path) throws IOException {
+		byte[] encoded = Files.readAllBytes(path);
+		return new String(encoded, Charset.defaultCharset());
+	}
+
+	private List<File> findNoticeFiles(File root) throws IOException {
+		return Files.walk(root.toPath())
+			.filter(file -> {
+				int nameCount = file.getNameCount();
+				return file.getName(nameCount - 1).toString().equals("NOTICE")
+					&& file.getName(nameCount - 2).toString().equals("META-INF")
+					&& file.getName(nameCount - 3).toString().equals("resources");
+			})
+			.map(Path::toFile)
+			.collect(Collectors.toList());
+	}
+
+	private Multimap<String, IncludedDependency> parseModulesFromBuildResult(File buildResult) throws IOException {
+		Multimap<String, IncludedDependency> result = ArrayListMultimap.create();
+		try (BufferedReader br = new BufferedReader(new FileReader(buildResult))) {

Review comment:
       If you were to use `Files.readAllLines()` you could use an indexed for loop, and parse each block within a single iteration, instead of having to keep state around to modify the behavior of the next one.

##########
File path: tools/ci/java-ci-tools/src/main/resources/modules-skipping-deployment.modulelist
##########
@@ -0,0 +1,43 @@
+################################################################################
+#  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.
+################################################################################
+
+# These modules are not deployed to maven central, despite their use of the shade plugin.
+
+flink-examples-streaming-twitter
+flink-examples-streaming-state-machine
+flink-examples-streaming-gcp-pubsub
+flink-s3-fs-base

Review comment:
       This may at some point bundle dependencies, so It would be better to explicitly check it.

##########
File path: tools/ci/maven-utils.sh
##########
@@ -22,11 +22,11 @@ function run_mvn {
 	fi
 
 	ARGS=$@
-	INVOCATION="$MVN_CMD $MVN_GLOBAL_OPTIONS $ARGS"
+	INVOCATION="${MVN_CMD} ${MVN_GLOBAL_OPTIONS} ${ARGS}"

Review comment:
       why are you adding braces here, but remove them below?

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Scanner;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), new File(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:3.1.1:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, File root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithShadedDependencies = parseModulesFromBuildResult(buildResult);

Review comment:
       ```suggestion
   		Multimap<String, IncludedDependency> modulesWithBundledDependencies = parseModulesFromBuildResult(buildResult);
   ```
   
   I'd like to avoid "shading" as a term because it is ambiguous (used both for describing the usage of a shade plugin, and relocating dependencies); "bundling" is more in line with our licensing guide.

##########
File path: tools/ci/java-ci-tools/src/main/resources/modules-skipping-deployment.modulelist
##########
@@ -0,0 +1,43 @@
+################################################################################
+#  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.
+################################################################################
+
+# These modules are not deployed to maven central, despite their use of the shade plugin.
+
+flink-examples-streaming-twitter
+flink-examples-streaming-state-machine

Review comment:
       this is bundled in the distribution and needs a correct notice.

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Scanner;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), new File(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:3.1.1:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");

Review comment:
       exclude the version from the pattern to reduce maintenance overhead

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Scanner;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), new File(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:3.1.1:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, File root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithShadedDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithShadedDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithShadedDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<File> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithShadedDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (File noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithShadedDependencies, noticeFile);
+		}
+
+		// find modules included in flink-dist

Review comment:
       ?

##########
File path: tools/azure-pipelines/jobs-template.yml
##########
@@ -229,7 +229,7 @@ jobs:
         IT_CASE_S3_BUCKET: $(SECRET_S3_BUCKET)
         IT_CASE_S3_ACCESS_KEY: $(SECRET_S3_ACCESS_KEY)
         IT_CASE_S3_SECRET_KEY: $(SECRET_S3_SECRET_KEY)
-      condition: not(eq(variables['SKIP'], '1'))
+      condition: and(succeeded(),not(eq(variables['SKIP'], '1')))

Review comment:
       how is this related?

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Scanner;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), new File(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:3.1.1:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, File root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithShadedDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithShadedDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithShadedDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<File> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithShadedDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (File noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithShadedDependencies, noticeFile);
+		}
+
+		// find modules included in flink-dist
+
+		return severeIssueCount;
+	}
+
+	private int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<File> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(File noticeFile) {
+		File moduleFile = noticeFile.getParentFile() // META-INF
+			.getParentFile() // resources
+			.getParentFile() // main
+			.getParentFile() // src
+			.getParentFile(); // <-- module name
+		return moduleFile.getName();
+	}
+
+	private int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, File noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		String noticeContents = readFile(noticeFile.toPath());
+		if (!noticeContents.startsWith(moduleName)) {
+			String firstLine = noticeContents.substring(0, noticeContents.indexOf('\n'));
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, firstLine);
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		try (BufferedReader br = new BufferedReader(new StringReader(noticeContents))) {
+			String line;
+			while ((line = br.readLine()) != null) {
+				Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+				if (noticeDependencyMatcher.find()) {
+					String groupId = noticeDependencyMatcher.group(1);
+					String artifactId = noticeDependencyMatcher.group(2);
+					String version = noticeDependencyMatcher.group(3);
+					if (groupId == null && artifactId == null && version == null) { // "bundles" case
+						groupId = noticeDependencyMatcher.group(5);
+						artifactId = noticeDependencyMatcher.group(6);
+						version = noticeDependencyMatcher.group(7);
+					}
+					IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+					if (!declaredDependencies.add(toAdd)) {
+						LOG.warn("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					}
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Set<IncludedDependency> expectedDependencies = new HashSet<>(modulesWithShadedDependencies.get(moduleName));
+		expectedDependencies.removeAll(declaredDependencies);
+		for (IncludedDependency missingDependency : expectedDependencies) {
+			LOG.error("Could not find dependency {} in NOTICE file {}", missingDependency, noticeFile);
+			severeIssueCount++;
+		}
+
+		if (!MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName)) {
+			// print all dependencies defined in NOTICE file, which were not expected
+			Set<IncludedDependency> excessDependencies = new HashSet<>(declaredDependencies);
+			excessDependencies.removeAll(modulesWithShadedDependencies.get(moduleName));
+			for (IncludedDependency excessDependency : excessDependencies) {
+				LOG.warn("Dependency {} is mentioned in NOTICE file {}, but is not expected there", excessDependency, noticeFile);
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static String readFile(Path path) throws IOException {
+		byte[] encoded = Files.readAllBytes(path);
+		return new String(encoded, Charset.defaultCharset());
+	}
+
+	private List<File> findNoticeFiles(File root) throws IOException {

Review comment:
       ```suggestion
   	private static List<File> findNoticeFiles(File root) throws IOException {
   ```

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Scanner;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), new File(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:3.1.1:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, File root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithShadedDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithShadedDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithShadedDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<File> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithShadedDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (File noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithShadedDependencies, noticeFile);
+		}
+
+		// find modules included in flink-dist
+
+		return severeIssueCount;
+	}
+
+	private int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<File> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(File noticeFile) {
+		File moduleFile = noticeFile.getParentFile() // META-INF
+			.getParentFile() // resources
+			.getParentFile() // main
+			.getParentFile() // src
+			.getParentFile(); // <-- module name
+		return moduleFile.getName();
+	}
+
+	private int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, File noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		String noticeContents = readFile(noticeFile.toPath());
+		if (!noticeContents.startsWith(moduleName)) {
+			String firstLine = noticeContents.substring(0, noticeContents.indexOf('\n'));
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, firstLine);
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		try (BufferedReader br = new BufferedReader(new StringReader(noticeContents))) {
+			String line;
+			while ((line = br.readLine()) != null) {
+				Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+				if (noticeDependencyMatcher.find()) {
+					String groupId = noticeDependencyMatcher.group(1);
+					String artifactId = noticeDependencyMatcher.group(2);
+					String version = noticeDependencyMatcher.group(3);
+					if (groupId == null && artifactId == null && version == null) { // "bundles" case
+						groupId = noticeDependencyMatcher.group(5);
+						artifactId = noticeDependencyMatcher.group(6);
+						version = noticeDependencyMatcher.group(7);
+					}
+					IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+					if (!declaredDependencies.add(toAdd)) {
+						LOG.warn("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					}
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Set<IncludedDependency> expectedDependencies = new HashSet<>(modulesWithShadedDependencies.get(moduleName));
+		expectedDependencies.removeAll(declaredDependencies);
+		for (IncludedDependency missingDependency : expectedDependencies) {
+			LOG.error("Could not find dependency {} in NOTICE file {}", missingDependency, noticeFile);
+			severeIssueCount++;
+		}
+
+		if (!MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName)) {
+			// print all dependencies defined in NOTICE file, which were not expected
+			Set<IncludedDependency> excessDependencies = new HashSet<>(declaredDependencies);
+			excessDependencies.removeAll(modulesWithShadedDependencies.get(moduleName));
+			for (IncludedDependency excessDependency : excessDependencies) {
+				LOG.warn("Dependency {} is mentioned in NOTICE file {}, but is not expected there", excessDependency, noticeFile);
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static String readFile(Path path) throws IOException {
+		byte[] encoded = Files.readAllBytes(path);
+		return new String(encoded, Charset.defaultCharset());
+	}
+
+	private List<File> findNoticeFiles(File root) throws IOException {
+		return Files.walk(root.toPath())
+			.filter(file -> {
+				int nameCount = file.getNameCount();
+				return file.getName(nameCount - 1).toString().equals("NOTICE")
+					&& file.getName(nameCount - 2).toString().equals("META-INF")
+					&& file.getName(nameCount - 3).toString().equals("resources");
+			})
+			.map(Path::toFile)
+			.collect(Collectors.toList());
+	}
+
+	private Multimap<String, IncludedDependency> parseModulesFromBuildResult(File buildResult) throws IOException {

Review comment:
       ```suggestion
   	private static Multimap<String, IncludedDependency> parseModulesFromBuildResult(File buildResult) throws IOException {
   ```

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Scanner;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), new File(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:3.1.1:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, File root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithShadedDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithShadedDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithShadedDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<File> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithShadedDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (File noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithShadedDependencies, noticeFile);
+		}
+
+		// find modules included in flink-dist
+
+		return severeIssueCount;
+	}
+
+	private int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<File> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(File noticeFile) {
+		File moduleFile = noticeFile.getParentFile() // META-INF
+			.getParentFile() // resources
+			.getParentFile() // main
+			.getParentFile() // src
+			.getParentFile(); // <-- module name
+		return moduleFile.getName();
+	}
+
+	private int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, File noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		String noticeContents = readFile(noticeFile.toPath());
+		if (!noticeContents.startsWith(moduleName)) {
+			String firstLine = noticeContents.substring(0, noticeContents.indexOf('\n'));
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, firstLine);
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		try (BufferedReader br = new BufferedReader(new StringReader(noticeContents))) {
+			String line;
+			while ((line = br.readLine()) != null) {
+				Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+				if (noticeDependencyMatcher.find()) {
+					String groupId = noticeDependencyMatcher.group(1);
+					String artifactId = noticeDependencyMatcher.group(2);
+					String version = noticeDependencyMatcher.group(3);
+					if (groupId == null && artifactId == null && version == null) { // "bundles" case
+						groupId = noticeDependencyMatcher.group(5);
+						artifactId = noticeDependencyMatcher.group(6);
+						version = noticeDependencyMatcher.group(7);
+					}
+					IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+					if (!declaredDependencies.add(toAdd)) {
+						LOG.warn("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					}
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Set<IncludedDependency> expectedDependencies = new HashSet<>(modulesWithShadedDependencies.get(moduleName));
+		expectedDependencies.removeAll(declaredDependencies);
+		for (IncludedDependency missingDependency : expectedDependencies) {
+			LOG.error("Could not find dependency {} in NOTICE file {}", missingDependency, noticeFile);
+			severeIssueCount++;
+		}
+
+		if (!MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName)) {
+			// print all dependencies defined in NOTICE file, which were not expected
+			Set<IncludedDependency> excessDependencies = new HashSet<>(declaredDependencies);
+			excessDependencies.removeAll(modulesWithShadedDependencies.get(moduleName));
+			for (IncludedDependency excessDependency : excessDependencies) {
+				LOG.warn("Dependency {} is mentioned in NOTICE file {}, but is not expected there", excessDependency, noticeFile);
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static String readFile(Path path) throws IOException {
+		byte[] encoded = Files.readAllBytes(path);

Review comment:
       any particular reason you are reading everything into a single string, instead of using `Files.readAllLines()`?

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Scanner;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), new File(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:3.1.1:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");

Review comment:
       I'd include an example for both pattern; it tends to make things a lot easier to understand.

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Scanner;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), new File(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:3.1.1:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, File root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithShadedDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithShadedDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithShadedDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<File> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithShadedDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (File noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithShadedDependencies, noticeFile);
+		}
+
+		// find modules included in flink-dist
+
+		return severeIssueCount;
+	}
+
+	private int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<File> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(File noticeFile) {
+		File moduleFile = noticeFile.getParentFile() // META-INF
+			.getParentFile() // resources
+			.getParentFile() // main
+			.getParentFile() // src
+			.getParentFile(); // <-- module name
+		return moduleFile.getName();
+	}
+
+	private int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, File noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		String noticeContents = readFile(noticeFile.toPath());
+		if (!noticeContents.startsWith(moduleName)) {
+			String firstLine = noticeContents.substring(0, noticeContents.indexOf('\n'));
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, firstLine);
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		try (BufferedReader br = new BufferedReader(new StringReader(noticeContents))) {
+			String line;
+			while ((line = br.readLine()) != null) {
+				Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+				if (noticeDependencyMatcher.find()) {
+					String groupId = noticeDependencyMatcher.group(1);
+					String artifactId = noticeDependencyMatcher.group(2);
+					String version = noticeDependencyMatcher.group(3);
+					if (groupId == null && artifactId == null && version == null) { // "bundles" case
+						groupId = noticeDependencyMatcher.group(5);
+						artifactId = noticeDependencyMatcher.group(6);
+						version = noticeDependencyMatcher.group(7);
+					}
+					IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+					if (!declaredDependencies.add(toAdd)) {
+						LOG.warn("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					}
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Set<IncludedDependency> expectedDependencies = new HashSet<>(modulesWithShadedDependencies.get(moduleName));
+		expectedDependencies.removeAll(declaredDependencies);
+		for (IncludedDependency missingDependency : expectedDependencies) {
+			LOG.error("Could not find dependency {} in NOTICE file {}", missingDependency, noticeFile);
+			severeIssueCount++;
+		}

Review comment:
       ```suggestion
   		Set<IncludedDependency> expectedDependencies = modulesWithShadedDependencies.get(moduleName);
   		for (IncludedDependency expectedDependency : expectedDependencies) {
   			if (!includedDependencies.contains(expectedDependency)) {
   				LOG.error("Could not find dependency {} in NOTICE file {}", expectedDependency, noticeFile);
   				severeIssueCount++;
   			}
   		}
   ```
   you can side-step the issue of not wanting to modify the colleted dependencies by just not removing any entries in the first palce.

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Scanner;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), new File(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:3.1.1:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, File root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithShadedDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithShadedDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithShadedDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<File> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithShadedDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (File noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithShadedDependencies, noticeFile);
+		}
+
+		// find modules included in flink-dist
+
+		return severeIssueCount;
+	}
+
+	private int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<File> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(File noticeFile) {
+		File moduleFile = noticeFile.getParentFile() // META-INF
+			.getParentFile() // resources
+			.getParentFile() // main
+			.getParentFile() // src
+			.getParentFile(); // <-- module name
+		return moduleFile.getName();
+	}
+
+	private int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, File noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		String noticeContents = readFile(noticeFile.toPath());
+		if (!noticeContents.startsWith(moduleName)) {
+			String firstLine = noticeContents.substring(0, noticeContents.indexOf('\n'));
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, firstLine);
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		try (BufferedReader br = new BufferedReader(new StringReader(noticeContents))) {
+			String line;
+			while ((line = br.readLine()) != null) {
+				Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+				if (noticeDependencyMatcher.find()) {
+					String groupId = noticeDependencyMatcher.group(1);
+					String artifactId = noticeDependencyMatcher.group(2);
+					String version = noticeDependencyMatcher.group(3);
+					if (groupId == null && artifactId == null && version == null) { // "bundles" case
+						groupId = noticeDependencyMatcher.group(5);
+						artifactId = noticeDependencyMatcher.group(6);
+						version = noticeDependencyMatcher.group(7);
+					}
+					IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+					if (!declaredDependencies.add(toAdd)) {
+						LOG.warn("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					}
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Set<IncludedDependency> expectedDependencies = new HashSet<>(modulesWithShadedDependencies.get(moduleName));
+		expectedDependencies.removeAll(declaredDependencies);
+		for (IncludedDependency missingDependency : expectedDependencies) {
+			LOG.error("Could not find dependency {} in NOTICE file {}", missingDependency, noticeFile);
+			severeIssueCount++;
+		}
+
+		if (!MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName)) {
+			// print all dependencies defined in NOTICE file, which were not expected
+			Set<IncludedDependency> excessDependencies = new HashSet<>(declaredDependencies);
+			excessDependencies.removeAll(modulesWithShadedDependencies.get(moduleName));

Review comment:
       same as above

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Scanner;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), new File(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:3.1.1:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, File root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithShadedDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithShadedDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithShadedDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<File> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithShadedDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (File noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithShadedDependencies, noticeFile);
+		}
+
+		// find modules included in flink-dist
+
+		return severeIssueCount;
+	}
+
+	private int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<File> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(File noticeFile) {
+		File moduleFile = noticeFile.getParentFile() // META-INF
+			.getParentFile() // resources
+			.getParentFile() // main
+			.getParentFile() // src
+			.getParentFile(); // <-- module name
+		return moduleFile.getName();
+	}
+
+	private int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, File noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		String noticeContents = readFile(noticeFile.toPath());
+		if (!noticeContents.startsWith(moduleName)) {
+			String firstLine = noticeContents.substring(0, noticeContents.indexOf('\n'));
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, firstLine);
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		try (BufferedReader br = new BufferedReader(new StringReader(noticeContents))) {
+			String line;
+			while ((line = br.readLine()) != null) {
+				Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+				if (noticeDependencyMatcher.find()) {
+					String groupId = noticeDependencyMatcher.group(1);
+					String artifactId = noticeDependencyMatcher.group(2);
+					String version = noticeDependencyMatcher.group(3);
+					if (groupId == null && artifactId == null && version == null) { // "bundles" case
+						groupId = noticeDependencyMatcher.group(5);
+						artifactId = noticeDependencyMatcher.group(6);
+						version = noticeDependencyMatcher.group(7);
+					}
+					IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+					if (!declaredDependencies.add(toAdd)) {
+						LOG.warn("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					}
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Set<IncludedDependency> expectedDependencies = new HashSet<>(modulesWithShadedDependencies.get(moduleName));
+		expectedDependencies.removeAll(declaredDependencies);
+		for (IncludedDependency missingDependency : expectedDependencies) {
+			LOG.error("Could not find dependency {} in NOTICE file {}", missingDependency, noticeFile);
+			severeIssueCount++;
+		}
+
+		if (!MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName)) {
+			// print all dependencies defined in NOTICE file, which were not expected
+			Set<IncludedDependency> excessDependencies = new HashSet<>(declaredDependencies);
+			excessDependencies.removeAll(modulesWithShadedDependencies.get(moduleName));
+			for (IncludedDependency excessDependency : excessDependencies) {
+				LOG.warn("Dependency {} is mentioned in NOTICE file {}, but is not expected there", excessDependency, noticeFile);
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static String readFile(Path path) throws IOException {
+		byte[] encoded = Files.readAllBytes(path);
+		return new String(encoded, Charset.defaultCharset());
+	}
+
+	private List<File> findNoticeFiles(File root) throws IOException {
+		return Files.walk(root.toPath())
+			.filter(file -> {
+				int nameCount = file.getNameCount();
+				return file.getName(nameCount - 1).toString().equals("NOTICE")
+					&& file.getName(nameCount - 2).toString().equals("META-INF")
+					&& file.getName(nameCount - 3).toString().equals("resources");
+			})
+			.map(Path::toFile)
+			.collect(Collectors.toList());
+	}
+
+	private Multimap<String, IncludedDependency> parseModulesFromBuildResult(File buildResult) throws IOException {
+		Multimap<String, IncludedDependency> result = ArrayListMultimap.create();
+		try (BufferedReader br = new BufferedReader(new FileReader(buildResult))) {
+			String line;
+			String currentShadeModule = null;
+			String currentDependencyCopyModule = null;
+			while ((line = br.readLine()) != null) {
+				Matcher nextShadeModuleMatcher = SHADE_NEXT_MODULE_PATTERN.matcher(line);
+				if (nextShadeModuleMatcher.find()) {
+					currentShadeModule = nextShadeModuleMatcher.group(2);
+				}
+
+				Matcher nextDependencyCopyModuleMatcher = DEPENDENCY_COPY_NEXT_MODULE_PATTERN.matcher(line);
+				if (nextDependencyCopyModuleMatcher.find()) {
+					currentDependencyCopyModule = nextDependencyCopyModuleMatcher.group(1);
+				}
+
+				if (currentShadeModule != null) {
+					Matcher includeMatcher = SHADE_INCLUDE_MODULE_PATTERN.matcher(line);
+					if (includeMatcher.find()) {
+						String groupId = includeMatcher.group(1);
+						String artifactId = includeMatcher.group(2);
+						String version = includeMatcher.group(3);
+						if (!"org.apache.flink".equals(groupId)) {
+							result.put(currentShadeModule, IncludedDependency.create(groupId, artifactId, version));
+						}
+					}
+				}
+
+				if (currentDependencyCopyModule != null) {
+					Matcher copyMatcher = DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN.matcher(line);
+					if (copyMatcher.find()) {
+						String groupId = copyMatcher.group(1);
+						String artifactId = copyMatcher.group(2);
+						String version = copyMatcher.group(3);
+						if (!"org.apache.flink".equals(groupId)) {
+							result.put(currentDependencyCopyModule, IncludedDependency.create(groupId, artifactId, version));
+						}
+					}
+				}
+				if (line.contains("Replacing original artifact with shaded artifact")) {
+					currentShadeModule = null;
+				}
+				if (line.contains("Copying")) {
+					currentDependencyCopyModule = null;
+				}
+			}
+		}
+		return result;
+	}
+
+	private static List<String> loadFromResources(String fileName) {
+		List<String> res = new ArrayList<>();
+		try (InputStream in = LicenseChecker.class.getResourceAsStream("/" + fileName)) {
+			try (Scanner scanner = new Scanner(in)) {
+				while (scanner.hasNext()) {
+					String line = scanner.nextLine();
+					if (!line.startsWith("#")){
+						res.add(line);
+					}
+				}
+			}
+		} catch (IOException e) {
+			LOG.warn("Error while loading resource", e);
+		}

Review comment:
       I'm not sure if this works, but you could try
   
   ```
   Path resource = LicenseChecker.class.getResource("/" + filename).toPath();
   return Files.readAllLines(resource).stream().filter(line -> !line.startsWith("#")).collect(Collectors.toList());
   ```

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Scanner;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), new File(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:3.1.1:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, File root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithShadedDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithShadedDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithShadedDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<File> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithShadedDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (File noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithShadedDependencies, noticeFile);
+		}
+
+		// find modules included in flink-dist
+
+		return severeIssueCount;
+	}
+
+	private int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<File> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(File noticeFile) {
+		File moduleFile = noticeFile.getParentFile() // META-INF
+			.getParentFile() // resources
+			.getParentFile() // main
+			.getParentFile() // src
+			.getParentFile(); // <-- module name
+		return moduleFile.getName();
+	}
+
+	private int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, File noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		String noticeContents = readFile(noticeFile.toPath());
+		if (!noticeContents.startsWith(moduleName)) {
+			String firstLine = noticeContents.substring(0, noticeContents.indexOf('\n'));
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, firstLine);
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		try (BufferedReader br = new BufferedReader(new StringReader(noticeContents))) {
+			String line;
+			while ((line = br.readLine()) != null) {
+				Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+				if (noticeDependencyMatcher.find()) {
+					String groupId = noticeDependencyMatcher.group(1);
+					String artifactId = noticeDependencyMatcher.group(2);
+					String version = noticeDependencyMatcher.group(3);
+					if (groupId == null && artifactId == null && version == null) { // "bundles" case
+						groupId = noticeDependencyMatcher.group(5);
+						artifactId = noticeDependencyMatcher.group(6);
+						version = noticeDependencyMatcher.group(7);
+					}
+					IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+					if (!declaredDependencies.add(toAdd)) {
+						LOG.warn("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					}
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Set<IncludedDependency> expectedDependencies = new HashSet<>(modulesWithShadedDependencies.get(moduleName));
+		expectedDependencies.removeAll(declaredDependencies);
+		for (IncludedDependency missingDependency : expectedDependencies) {
+			LOG.error("Could not find dependency {} in NOTICE file {}", missingDependency, noticeFile);
+			severeIssueCount++;
+		}
+
+		if (!MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName)) {
+			// print all dependencies defined in NOTICE file, which were not expected
+			Set<IncludedDependency> excessDependencies = new HashSet<>(declaredDependencies);
+			excessDependencies.removeAll(modulesWithShadedDependencies.get(moduleName));
+			for (IncludedDependency excessDependency : excessDependencies) {
+				LOG.warn("Dependency {} is mentioned in NOTICE file {}, but is not expected there", excessDependency, noticeFile);
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static String readFile(Path path) throws IOException {
+		byte[] encoded = Files.readAllBytes(path);
+		return new String(encoded, Charset.defaultCharset());
+	}
+
+	private List<File> findNoticeFiles(File root) throws IOException {
+		return Files.walk(root.toPath())
+			.filter(file -> {
+				int nameCount = file.getNameCount();
+				return file.getName(nameCount - 1).toString().equals("NOTICE")

Review comment:
       maybe invert the list of conditions

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Scanner;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), new File(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:3.1.1:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, File root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithShadedDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithShadedDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithShadedDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<File> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithShadedDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (File noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithShadedDependencies, noticeFile);
+		}
+
+		// find modules included in flink-dist
+
+		return severeIssueCount;
+	}
+
+	private int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<File> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(File noticeFile) {
+		File moduleFile = noticeFile.getParentFile() // META-INF
+			.getParentFile() // resources
+			.getParentFile() // main
+			.getParentFile() // src
+			.getParentFile(); // <-- module name
+		return moduleFile.getName();

Review comment:
       ```suggestion
   	private static String getModuleFromNoticeFile(Path noticeFile) {
   		File moduleDirectory = noticeFile.getParent() // META-INF
   			.getParent() // resources
   			.getParent() // main
   			.getParent() // src
   			.getParent(); // <-- module name
   		return moduleDirectory.getFileName();
   ```
   With this change we should be able to use `Path` everywhere, so we can stop converting things everywhere.

##########
File path: flink-connectors/flink-sql-connector-hbase-1.4/pom.xml
##########
@@ -87,6 +87,7 @@ under the License.
 									<exclude>org.apache.hbase:hbase-metrics*</exclude>
 									<exclude>org.apache.hbase:hbase-server*</exclude>
 									<exclude>org.apache.hbase:hbase-hadoop*-compat</exclude>
+									<exclude>org.apache.hbase:hbase-common:jar:tests</exclude>

Review comment:
       If they are only test classes then I'm wondering why this shows up in the dependency tree, and an exclusion or explicit test dependency in the connector module might be more appropriate.

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -120,87 +122,100 @@ private int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modul
 		return severeIssueCount;
 	}
 
-	private static String getModuleFromNoticeFile(File noticeFile) {
-		File moduleFile = noticeFile.getParentFile() // META-INF
-			.getParentFile() // resources
-			.getParentFile() // main
-			.getParentFile() // src
-			.getParentFile(); // <-- module name
-		return moduleFile.getName();
+	private static String getModuleFromNoticeFile(Path noticeFile) {
+		Path moduleDirectory = noticeFile.getParent() // META-INF
+			.getParent() // resources
+			.getParent() // main
+			.getParent() // src
+			.getParent(); // <-- module name
+		return moduleDirectory.getFileName().toString();
 	}
 
-	private int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, File noticeFile) throws IOException {
+	private static int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, Path noticeFile) throws IOException {
 		int severeIssueCount = 0;
 		String moduleName = getModuleFromNoticeFile(noticeFile);
 
 		// 1st line contains module name
-		String noticeContents = readFile(noticeFile.toPath());
-		if (!noticeContents.startsWith(moduleName)) {
-			String firstLine = noticeContents.substring(0, noticeContents.indexOf('\n'));
-			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, firstLine);
+		List<String> noticeContents = Files.readAllLines(noticeFile);
+
+		if (noticeContents.isEmpty()) {
+			LOG.error("Notice file empty {}", noticeFile);
+			severeIssueCount++;
+		}
+
+		// first line must be the module name.
+		if (!noticeContents.get(0).equals(moduleName)) {
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, noticeContents.get(0));
 		}
 
 		// collect all declared dependencies from NOTICE file
 		Set<IncludedDependency> declaredDependencies = new HashSet<>();
-		try (BufferedReader br = new BufferedReader(new StringReader(noticeContents))) {
-			String line;
-			while ((line = br.readLine()) != null) {
-				Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
-				if (noticeDependencyMatcher.find()) {
-					String groupId = noticeDependencyMatcher.group(1);
-					String artifactId = noticeDependencyMatcher.group(2);
-					String version = noticeDependencyMatcher.group(3);
-					if (groupId == null && artifactId == null && version == null) { // "bundles" case
-						groupId = noticeDependencyMatcher.group(5);
-						artifactId = noticeDependencyMatcher.group(6);
-						version = noticeDependencyMatcher.group(7);
-					}
-					IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
-					if (!declaredDependencies.add(toAdd)) {
-						LOG.warn("Dependency {} has been declared twice in module {}", toAdd, moduleName);
-					}
+		for (String line : noticeContents) {
+			Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+			if (noticeDependencyMatcher.find()) {
+				String groupId = noticeDependencyMatcher.group(1);
+				String artifactId = noticeDependencyMatcher.group(2);
+				String version = noticeDependencyMatcher.group(3);
+				if (groupId == null && artifactId == null && version == null) { // "bundles" case
+					groupId = noticeDependencyMatcher.group(5);
+					artifactId = noticeDependencyMatcher.group(6);
+					version = noticeDependencyMatcher.group(7);
+				}
+				IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+				if (!declaredDependencies.add(toAdd)) {
+					LOG.error("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					severeIssueCount++;
 				}
 			}
 		}
 		// print all dependencies missing from NOTICE file
-		Set<IncludedDependency> expectedDependencies = new HashSet<>(modulesWithShadedDependencies.get(moduleName));
-		expectedDependencies.removeAll(declaredDependencies);
-		for (IncludedDependency missingDependency : expectedDependencies) {
-			LOG.error("Could not find dependency {} in NOTICE file {}", missingDependency, noticeFile);
-			severeIssueCount++;
+		Collection<IncludedDependency> expectedDependencies = modulesWithShadedDependencies.get(moduleName);
+		for (IncludedDependency expectedDependency : expectedDependencies) {
+			if (!declaredDependencies.contains(expectedDependency)) {
+				LOG.error("Could not find dependency {} in NOTICE file {}", expectedDependency, noticeFile);
+				severeIssueCount++;
+			}
 		}
 
-		if (!MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName)) {
-			// print all dependencies defined in NOTICE file, which were not expected
-			Set<IncludedDependency> excessDependencies = new HashSet<>(declaredDependencies);
-			excessDependencies.removeAll(modulesWithShadedDependencies.get(moduleName));
-			for (IncludedDependency excessDependency : excessDependencies) {
-				LOG.warn("Dependency {} is mentioned in NOTICE file {}, but is not expected there", excessDependency, noticeFile);
+		boolean moduleDefinesExcessDependencies = MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName);
+
+		// print all dependencies defined in NOTICE file, which were not expected
+		Set<IncludedDependency> excessDependencies = new HashSet<>(declaredDependencies);

Review comment:
       can be removed

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -251,21 +266,19 @@ private static String readFile(Path path) throws IOException {
 	}
 
 	private static List<String> loadFromResources(String fileName) {
-		List<String> res = new ArrayList<>();
-		try (InputStream in = LicenseChecker.class.getResourceAsStream("/" + fileName)) {
-			try (Scanner scanner = new Scanner(in)) {
-				while (scanner.hasNext()) {
-					String line = scanner.nextLine();
-					if (!line.startsWith("#")){
-						res.add(line);
-					}
-				}
-			}
-		} catch (IOException e) {
+		try {
+			Path resource = Paths.get(LicenseChecker.class.getResource("/" + fileName).toURI());
+			List<String> result = Files
+				.readAllLines(resource)
+				.stream()
+				.filter(line -> !line.startsWith("#") && !line.isEmpty())
+				.collect(Collectors.toList());
+			LOG.debug("Loaded {} items from resource {}", result.size(), fileName);
+			return result;
+		} catch (Throwable e) {
 			LOG.warn("Error while loading resource", e);

Review comment:
       We should fail hard here because it implies some programming error, and it could mislead people into thinking that something is wrong with the licensing.

##########
File path: tools/ci/java-ci-tools/pom.xml
##########
@@ -23,49 +23,32 @@ under the License.
 	<modelVersion>4.0.0</modelVersion>
 
 	<parent>
-		<groupId>org.apache</groupId>
-		<artifactId>apache</artifactId>
-		<version>18</version>
+		<groupId>org.apache.flink</groupId>
+		<artifactId>flink-parent</artifactId>
+		<version>1.12-SNAPSHOT</version>
 	</parent>
 
-	<groupId>org.apache.flink</groupId>
 	<artifactId>java-ci-tools</artifactId>
 	<version>1.12-SNAPSHOT</version>
 	<name>Flink : Tools : CI : Java</name>
 
-	<packaging>jar</packaging>
-	<properties>
-		<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-		<maven.compiler.source>1.8</maven.compiler.source>
-		<maven.compiler.target>1.8</maven.compiler.target>
-		<log4j.version>2.12.1</log4j.version>
-	</properties>
-
 	<dependencies>
 		<dependency>
 			<groupId>com.google.guava</groupId>
 			<artifactId>guava</artifactId>
 			<version>30.0-jre</version>
 		</dependency>
-		<dependency>
-			<groupId>org.slf4j</groupId>
-			<artifactId>slf4j-api</artifactId>
-			<version>1.7.30</version>
-		</dependency>
 		<dependency>
 			<groupId>org.apache.logging.log4j</groupId>
 			<artifactId>log4j-slf4j-impl</artifactId>
-			<version>${log4j.version}</version>
 		</dependency>
 		<dependency>
 			<groupId>org.apache.logging.log4j</groupId>
 			<artifactId>log4j-api</artifactId>
-			<version>${log4j.version}</version>

Review comment:
       Don't we have to explicitly set these as compile dependencies.

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+//CHECKSTYLE.OFF: regexp|imports
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+//CHECKSTYLE.ON: regexp|imports
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), Paths.get(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:[^:]+:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	// Examples:
+	// "- org.apache.htrace:htrace-core:3.1.0-incubating"
+	// or
+	// "This project bundles "net.jcip:jcip-annotations:1.0".
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, Path root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithBundledDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithBundledDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithBundledDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<Path> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithBundledDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (Path noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithBundledDependencies, noticeFile);
+		}
+
+		return severeIssueCount;
+	}
+
+	private static int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<Path> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(Path noticeFile) {
+		Path moduleDirectory = noticeFile.getParent() // META-INF
+			.getParent() // resources
+			.getParent() // main
+			.getParent() // src
+			.getParent(); // <-- module name
+		return moduleDirectory.getFileName().toString();
+	}
+
+	private static int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, Path noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		List<String> noticeContents = Files.readAllLines(noticeFile);
+
+		if (noticeContents.isEmpty()) {
+			LOG.error("Notice file empty {}", noticeFile);
+			severeIssueCount++;
+		}
+
+		// first line must be the module name.
+		if (!noticeContents.get(0).equals(moduleName)) {
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, noticeContents.get(0));
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		for (String line : noticeContents) {
+			Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+			if (noticeDependencyMatcher.find()) {
+				String groupId = noticeDependencyMatcher.group(1);
+				String artifactId = noticeDependencyMatcher.group(2);
+				String version = noticeDependencyMatcher.group(3);
+				if (groupId == null && artifactId == null && version == null) { // "bundles" case
+					groupId = noticeDependencyMatcher.group(5);
+					artifactId = noticeDependencyMatcher.group(6);
+					version = noticeDependencyMatcher.group(7);
+				}
+				IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+				if (!declaredDependencies.add(toAdd)) {
+					LOG.error("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					severeIssueCount++;
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Collection<IncludedDependency> expectedDependencies = modulesWithShadedDependencies.get(moduleName);
+		for (IncludedDependency expectedDependency : expectedDependencies) {
+			if (!declaredDependencies.contains(expectedDependency)) {
+				LOG.error("Could not find dependency {} in NOTICE file {}", expectedDependency, noticeFile);
+				severeIssueCount++;
+			}
+		}
+
+		boolean moduleDefinesExcessDependencies = MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName);
+
+		// print all dependencies defined in NOTICE file, which were not expected
+		Set<IncludedDependency> excessDependencies = new HashSet<>(declaredDependencies);
+		excessDependencies.removeAll(modulesWithShadedDependencies.get(moduleName));
+		for (IncludedDependency declaredDependency : declaredDependencies) {
+			if (!expectedDependencies.contains(declaredDependency)) {
+				if (moduleDefinesExcessDependencies) {
+					LOG.debug(
+						"Dependency {} is mentioned in NOTICE file {}, but was not mentioned by the build output as a bundled dependency",
+						declaredDependency,
+						noticeFile);
+				} else {
+					LOG.warn(
+						"Dependency {} is mentioned in NOTICE file {}, but is not expected there",
+						declaredDependency,
+						noticeFile);
+				}
+
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static List<Path> findNoticeFiles(Path root) throws IOException {
+		return Files.walk(root)
+			.filter(file -> {
+				int nameCount = file.getNameCount();
+				return file.getName(nameCount - 3).toString().equals("resources")
+					&& file.getName(nameCount - 2).toString().equals("META-INF")
+					&& file.getName(nameCount - 1).toString().equals("NOTICE");
+			})
+			.collect(Collectors.toList());
+	}
+
+	private static Multimap<String, IncludedDependency> parseModulesFromBuildResult(File buildResult) throws IOException {
+		Multimap<String, IncludedDependency> result = ArrayListMultimap.create();
+
+		try (BufferedReader br = new BufferedReader(new FileReader(buildResult))) {
+			String line;
+			String currentShadeModule = null;
+			String currentDependencyCopyModule = null;
+			while ((line = br.readLine()) != null) {

Review comment:
       ```suggestion
   		try (Stream<String> lines = Files.lines(buildResult.toPath()) {
   			String currentShadeModule = null;
   			String currentDependencyCopyModule = null;
   			for (String line : lines) {
   ```

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Scanner;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), new File(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:3.1.1:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, File root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithShadedDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithShadedDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithShadedDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<File> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithShadedDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (File noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithShadedDependencies, noticeFile);
+		}
+
+		// find modules included in flink-dist
+
+		return severeIssueCount;
+	}
+
+	private int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<File> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(File noticeFile) {
+		File moduleFile = noticeFile.getParentFile() // META-INF
+			.getParentFile() // resources
+			.getParentFile() // main
+			.getParentFile() // src
+			.getParentFile(); // <-- module name
+		return moduleFile.getName();
+	}
+
+	private int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, File noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		String noticeContents = readFile(noticeFile.toPath());
+		if (!noticeContents.startsWith(moduleName)) {
+			String firstLine = noticeContents.substring(0, noticeContents.indexOf('\n'));
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, firstLine);
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		try (BufferedReader br = new BufferedReader(new StringReader(noticeContents))) {
+			String line;
+			while ((line = br.readLine()) != null) {
+				Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+				if (noticeDependencyMatcher.find()) {
+					String groupId = noticeDependencyMatcher.group(1);
+					String artifactId = noticeDependencyMatcher.group(2);
+					String version = noticeDependencyMatcher.group(3);
+					if (groupId == null && artifactId == null && version == null) { // "bundles" case
+						groupId = noticeDependencyMatcher.group(5);
+						artifactId = noticeDependencyMatcher.group(6);
+						version = noticeDependencyMatcher.group(7);
+					}
+					IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+					if (!declaredDependencies.add(toAdd)) {
+						LOG.warn("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					}
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Set<IncludedDependency> expectedDependencies = new HashSet<>(modulesWithShadedDependencies.get(moduleName));
+		expectedDependencies.removeAll(declaredDependencies);
+		for (IncludedDependency missingDependency : expectedDependencies) {
+			LOG.error("Could not find dependency {} in NOTICE file {}", missingDependency, noticeFile);
+			severeIssueCount++;
+		}
+
+		if (!MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName)) {
+			// print all dependencies defined in NOTICE file, which were not expected
+			Set<IncludedDependency> excessDependencies = new HashSet<>(declaredDependencies);
+			excessDependencies.removeAll(modulesWithShadedDependencies.get(moduleName));
+			for (IncludedDependency excessDependency : excessDependencies) {
+				LOG.warn("Dependency {} is mentioned in NOTICE file {}, but is not expected there", excessDependency, noticeFile);
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static String readFile(Path path) throws IOException {
+		byte[] encoded = Files.readAllBytes(path);
+		return new String(encoded, Charset.defaultCharset());
+	}
+
+	private List<File> findNoticeFiles(File root) throws IOException {
+		return Files.walk(root.toPath())
+			.filter(file -> {
+				int nameCount = file.getNameCount();
+				return file.getName(nameCount - 1).toString().equals("NOTICE")
+					&& file.getName(nameCount - 2).toString().equals("META-INF")
+					&& file.getName(nameCount - 3).toString().equals("resources");
+			})
+			.map(Path::toFile)
+			.collect(Collectors.toList());
+	}
+
+	private Multimap<String, IncludedDependency> parseModulesFromBuildResult(File buildResult) throws IOException {
+		Multimap<String, IncludedDependency> result = ArrayListMultimap.create();
+		try (BufferedReader br = new BufferedReader(new FileReader(buildResult))) {

Review comment:
       Fair enough. (Note that you could use Files.lines() instead of a BufferedReader, which should make it overall a tiny bit simpler)

##########
File path: flink-connectors/flink-connector-hive/src/main/resources/META-INF/NOTICE
##########
@@ -7,7 +7,6 @@ The Apache Software Foundation (http://www.apache.org/).
 This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt)
 
 - org.apache.parquet:parquet-hadoop:1.11.1
-- org.apache.parquet:parquet-format:2.4.0

Review comment:
       oki. let's remove it then, but maybe ping one of the hive guys to double check that the packaging is correct.

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,320 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringReader;
+import java.nio.charset.Charset;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Scanner;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), new File(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:3.1.1:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, File root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithShadedDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithShadedDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithShadedDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<File> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithShadedDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (File noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithShadedDependencies, noticeFile);
+		}
+
+		// find modules included in flink-dist
+
+		return severeIssueCount;
+	}
+
+	private int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<File> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(File noticeFile) {
+		File moduleFile = noticeFile.getParentFile() // META-INF
+			.getParentFile() // resources
+			.getParentFile() // main
+			.getParentFile() // src
+			.getParentFile(); // <-- module name
+		return moduleFile.getName();
+	}
+
+	private int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, File noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		String noticeContents = readFile(noticeFile.toPath());
+		if (!noticeContents.startsWith(moduleName)) {
+			String firstLine = noticeContents.substring(0, noticeContents.indexOf('\n'));
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, firstLine);
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		try (BufferedReader br = new BufferedReader(new StringReader(noticeContents))) {
+			String line;
+			while ((line = br.readLine()) != null) {
+				Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+				if (noticeDependencyMatcher.find()) {
+					String groupId = noticeDependencyMatcher.group(1);
+					String artifactId = noticeDependencyMatcher.group(2);
+					String version = noticeDependencyMatcher.group(3);
+					if (groupId == null && artifactId == null && version == null) { // "bundles" case
+						groupId = noticeDependencyMatcher.group(5);
+						artifactId = noticeDependencyMatcher.group(6);
+						version = noticeDependencyMatcher.group(7);
+					}
+					IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+					if (!declaredDependencies.add(toAdd)) {
+						LOG.warn("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					}
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Set<IncludedDependency> expectedDependencies = new HashSet<>(modulesWithShadedDependencies.get(moduleName));
+		expectedDependencies.removeAll(declaredDependencies);
+		for (IncludedDependency missingDependency : expectedDependencies) {
+			LOG.error("Could not find dependency {} in NOTICE file {}", missingDependency, noticeFile);
+			severeIssueCount++;
+		}
+
+		if (!MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName)) {
+			// print all dependencies defined in NOTICE file, which were not expected
+			Set<IncludedDependency> excessDependencies = new HashSet<>(declaredDependencies);
+			excessDependencies.removeAll(modulesWithShadedDependencies.get(moduleName));
+			for (IncludedDependency excessDependency : excessDependencies) {
+				LOG.warn("Dependency {} is mentioned in NOTICE file {}, but is not expected there", excessDependency, noticeFile);
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static String readFile(Path path) throws IOException {
+		byte[] encoded = Files.readAllBytes(path);
+		return new String(encoded, Charset.defaultCharset());
+	}
+
+	private List<File> findNoticeFiles(File root) throws IOException {
+		return Files.walk(root.toPath())
+			.filter(file -> {
+				int nameCount = file.getNameCount();
+				return file.getName(nameCount - 1).toString().equals("NOTICE")
+					&& file.getName(nameCount - 2).toString().equals("META-INF")
+					&& file.getName(nameCount - 3).toString().equals("resources");
+			})
+			.map(Path::toFile)
+			.collect(Collectors.toList());
+	}
+
+	private Multimap<String, IncludedDependency> parseModulesFromBuildResult(File buildResult) throws IOException {
+		Multimap<String, IncludedDependency> result = ArrayListMultimap.create();
+		try (BufferedReader br = new BufferedReader(new FileReader(buildResult))) {

Review comment:
       Fair enough. (Note that you could use Files.lines() instead of a BufferedReader, which should make it overall a tiny bit simpler, see https://github.com/apache/flink/pull/13796#discussion_r516604899)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-716686693


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "e58da69e4aeb75fddcdb08c1ae89c2bc9789902a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8322",
       "triggerID" : "e58da69e4aeb75fddcdb08c1ae89c2bc9789902a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * e58da69e4aeb75fddcdb08c1ae89c2bc9789902a Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8322) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-717967845


   How would you integrate such a notice file generating build step into the CI pipeline?
   
   -  I've quickly looked at building our own shade resource transformer, but it seems that we won't have the necessary information available. We would probably need to go even further and change more stuff in the shade plugin.
   
   - The current approach depends on the maven build output. We could write the NOTICE file changes directly when the tool runs. Similar to the documentation generator, contributors would need to run the tool locally to keep the NOTICE files up to date. I guess this approach would be preferred.
   
   Any other ideas?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-721710429


   Ah, sorry. My comments have been confusing. I've initially had a scope there, then removed it (probably even before opening the PR, that's why I assumed you are referring to an outdated version of the PR).
   
   Isn't the "compile" scope the default scope: http://maven.apache.org/guides/introduction/introduction-to-dependency-mechanism.html#Dependency_Scope
   The dependencyManagement section of flink-parent isn't changing the scope of the dependency.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517177573



##########
File path: flink-connectors/flink-sql-connector-hbase-1.4/pom.xml
##########
@@ -87,6 +87,7 @@ under the License.
 									<exclude>org.apache.hbase:hbase-metrics*</exclude>
 									<exclude>org.apache.hbase:hbase-server*</exclude>
 									<exclude>org.apache.hbase:hbase-hadoop*-compat</exclude>
+									<exclude>org.apache.hbase:hbase-common:jar:tests</exclude>

Review comment:
       This is the dependency tree:
   ```
   [INFO] org.apache.flink:flink-sql-connector-hbase-1.4_2.11:jar:1.12-SNAPSHOT
   [INFO] +- org.apache.flink:flink-connector-hbase-1.4_2.11:jar:1.12-SNAPSHOT:compile
   [INFO] |  \- org.apache.flink:flink-connector-hbase-base_2.11:jar:1.12-SNAPSHOT:compile
   [INFO] |     +- org.apache.hbase:hbase-server:jar:1.4.3:compile
   [INFO] |     |  +- org.apache.hbase:hbase-common:jar:1.4.3:compile
   [INFO] |     |  |  +- org.apache.avro:avro:jar:1.10.0:compile
   [INFO] |     |  |  |  +- com.fasterxml.jackson.core:jackson-core:jar:2.10.1:compile
   [INFO] |     |  |  |  \- com.fasterxml.jackson.core:jackson-databind:jar:2.10.1:compile
   [INFO] |     |  |  |     \- com.fasterxml.jackson.core:jackson-annotations:jar:2.10.1:compile
   [INFO] |     |  |  \- com.github.stephenc.findbugs:findbugs-annotations:jar:1.3.9-1:compile
   [INFO] |     |  +- org.apache.hbase:hbase-protocol:jar:1.4.3:compile
   [INFO] |     |  +- org.apache.hbase:hbase-procedure:jar:1.4.3:compile
   [INFO] |     |  |  \- org.apache.hbase:hbase-common:jar:tests:1.4.3:compile
   ```
   
   I can not add an exclude on hbase-server, because I can not exclude based on the `tests` classifier.
   
   The other question is: why are we depending on `hbase-server`. It would probably be fine to depend on `hbase-common` and `hbase-client` (just judging from the names). I quickly tried that, but it lead to all kinds of problems with the tests (change is in https://github.com/rmetzger/flink/commit/07b60002da6e75b7d060d23f9c5ab571d34fb9b9) 
   How about working with the shade exclude for now, but filing a ticket, to solve this properly in the next release cycle?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517214200



##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+//CHECKSTYLE.OFF: regexp|imports
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+//CHECKSTYLE.ON: regexp|imports
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), Paths.get(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:[^:]+:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	// Examples:
+	// "- org.apache.htrace:htrace-core:3.1.0-incubating"
+	// or
+	// "This project bundles "net.jcip:jcip-annotations:1.0".
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, Path root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithBundledDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithBundledDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithBundledDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<Path> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithBundledDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (Path noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithBundledDependencies, noticeFile);
+		}
+
+		return severeIssueCount;
+	}
+
+	private static int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<Path> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(Path noticeFile) {
+		Path moduleDirectory = noticeFile.getParent() // META-INF
+			.getParent() // resources
+			.getParent() // main
+			.getParent() // src
+			.getParent(); // <-- module name
+		return moduleDirectory.getFileName().toString();
+	}
+
+	private static int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, Path noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		List<String> noticeContents = Files.readAllLines(noticeFile);
+
+		if (noticeContents.isEmpty()) {
+			LOG.error("Notice file empty {}", noticeFile);
+			severeIssueCount++;
+		}
+
+		// first line must be the module name.
+		if (!noticeContents.get(0).equals(moduleName)) {
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, noticeContents.get(0));
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		for (String line : noticeContents) {
+			Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+			if (noticeDependencyMatcher.find()) {
+				String groupId = noticeDependencyMatcher.group(1);
+				String artifactId = noticeDependencyMatcher.group(2);
+				String version = noticeDependencyMatcher.group(3);
+				if (groupId == null && artifactId == null && version == null) { // "bundles" case
+					groupId = noticeDependencyMatcher.group(5);
+					artifactId = noticeDependencyMatcher.group(6);
+					version = noticeDependencyMatcher.group(7);
+				}
+				IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+				if (!declaredDependencies.add(toAdd)) {
+					LOG.error("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					severeIssueCount++;
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Collection<IncludedDependency> expectedDependencies = modulesWithShadedDependencies.get(moduleName);
+		for (IncludedDependency expectedDependency : expectedDependencies) {
+			if (!declaredDependencies.contains(expectedDependency)) {
+				LOG.error("Could not find dependency {} in NOTICE file {}", expectedDependency, noticeFile);
+				severeIssueCount++;
+			}
+		}
+
+		boolean moduleDefinesExcessDependencies = MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName);
+
+		// print all dependencies defined in NOTICE file, which were not expected
+		for (IncludedDependency declaredDependency : declaredDependencies) {
+			if (!expectedDependencies.contains(declaredDependency)) {
+				if (moduleDefinesExcessDependencies) {
+					LOG.debug(
+						"Dependency {} is mentioned in NOTICE file {}, but was not mentioned by the build output as a bundled dependency",
+						declaredDependency,
+						noticeFile);
+				} else {
+					LOG.warn(
+						"Dependency {} is mentioned in NOTICE file {}, but is not expected there",
+						declaredDependency,
+						noticeFile);
+				}
+
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static List<Path> findNoticeFiles(Path root) throws IOException {
+		return Files.walk(root)
+			.filter(file -> {
+				int nameCount = file.getNameCount();
+				return file.getName(nameCount - 3).toString().equals("resources")
+					&& file.getName(nameCount - 2).toString().equals("META-INF")
+					&& file.getName(nameCount - 1).toString().equals("NOTICE");
+			})
+			.collect(Collectors.toList());
+	}
+
+	private static Multimap<String, IncludedDependency> parseModulesFromBuildResult(File buildResult) throws IOException {
+		Multimap<String, IncludedDependency> result = ArrayListMultimap.create();
+
+		try (Stream<String> lines = Files.lines(buildResult.toPath())) {
+			//String line;
+			String currentShadeModule = null;
+			String currentDependencyCopyModule = null;
+			for (String line : (Iterable<String>) lines::iterator) {

Review comment:
       I'm really not sure if it's worth going into this rabbit hole for a tiny syntactic improvement in an internal utility




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-721758994


   Thanks!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-721684343


   I've addressed this comment a while ago. Maybe you are looking at old commits?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517211912



##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+//CHECKSTYLE.OFF: regexp|imports
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+//CHECKSTYLE.ON: regexp|imports
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), Paths.get(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:[^:]+:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	// Examples:
+	// "- org.apache.htrace:htrace-core:3.1.0-incubating"
+	// or
+	// "This project bundles "net.jcip:jcip-annotations:1.0".
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, Path root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithBundledDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithBundledDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithBundledDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<Path> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithBundledDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (Path noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithBundledDependencies, noticeFile);
+		}
+
+		return severeIssueCount;
+	}
+
+	private static int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<Path> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(Path noticeFile) {
+		Path moduleDirectory = noticeFile.getParent() // META-INF
+			.getParent() // resources
+			.getParent() // main
+			.getParent() // src
+			.getParent(); // <-- module name
+		return moduleDirectory.getFileName().toString();
+	}
+
+	private static int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, Path noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		List<String> noticeContents = Files.readAllLines(noticeFile);
+
+		if (noticeContents.isEmpty()) {
+			LOG.error("Notice file empty {}", noticeFile);
+			severeIssueCount++;
+		}
+
+		// first line must be the module name.
+		if (!noticeContents.get(0).equals(moduleName)) {
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, noticeContents.get(0));
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		for (String line : noticeContents) {
+			Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+			if (noticeDependencyMatcher.find()) {
+				String groupId = noticeDependencyMatcher.group(1);
+				String artifactId = noticeDependencyMatcher.group(2);
+				String version = noticeDependencyMatcher.group(3);
+				if (groupId == null && artifactId == null && version == null) { // "bundles" case
+					groupId = noticeDependencyMatcher.group(5);
+					artifactId = noticeDependencyMatcher.group(6);
+					version = noticeDependencyMatcher.group(7);
+				}
+				IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+				if (!declaredDependencies.add(toAdd)) {
+					LOG.error("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					severeIssueCount++;
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Collection<IncludedDependency> expectedDependencies = modulesWithShadedDependencies.get(moduleName);
+		for (IncludedDependency expectedDependency : expectedDependencies) {
+			if (!declaredDependencies.contains(expectedDependency)) {
+				LOG.error("Could not find dependency {} in NOTICE file {}", expectedDependency, noticeFile);
+				severeIssueCount++;
+			}
+		}
+
+		boolean moduleDefinesExcessDependencies = MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName);
+
+		// print all dependencies defined in NOTICE file, which were not expected
+		for (IncludedDependency declaredDependency : declaredDependencies) {
+			if (!expectedDependencies.contains(declaredDependency)) {
+				if (moduleDefinesExcessDependencies) {
+					LOG.debug(
+						"Dependency {} is mentioned in NOTICE file {}, but was not mentioned by the build output as a bundled dependency",
+						declaredDependency,
+						noticeFile);
+				} else {
+					LOG.warn(
+						"Dependency {} is mentioned in NOTICE file {}, but is not expected there",
+						declaredDependency,
+						noticeFile);
+				}
+
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static List<Path> findNoticeFiles(Path root) throws IOException {
+		return Files.walk(root)
+			.filter(file -> {
+				int nameCount = file.getNameCount();
+				return file.getName(nameCount - 3).toString().equals("resources")
+					&& file.getName(nameCount - 2).toString().equals("META-INF")
+					&& file.getName(nameCount - 1).toString().equals("NOTICE");
+			})
+			.collect(Collectors.toList());
+	}
+
+	private static Multimap<String, IncludedDependency> parseModulesFromBuildResult(File buildResult) throws IOException {
+		Multimap<String, IncludedDependency> result = ArrayListMultimap.create();
+
+		try (Stream<String> lines = Files.lines(buildResult.toPath())) {
+			//String line;
+			String currentShadeModule = null;
+			String currentDependencyCopyModule = null;
+			for (String line : (Iterable<String>) lines::iterator) {
+				Matcher nextShadeModuleMatcher = SHADE_NEXT_MODULE_PATTERN.matcher(line);
+				if (nextShadeModuleMatcher.find()) {
+					currentShadeModule = nextShadeModuleMatcher.group(2);
+				}
+
+				Matcher nextDependencyCopyModuleMatcher = DEPENDENCY_COPY_NEXT_MODULE_PATTERN.matcher(line);
+				if (nextDependencyCopyModuleMatcher.find()) {
+					currentDependencyCopyModule = nextDependencyCopyModuleMatcher.group(1);
+				}
+
+				if (currentShadeModule != null) {
+					Matcher includeMatcher = SHADE_INCLUDE_MODULE_PATTERN.matcher(line);
+					if (includeMatcher.find()) {
+						String groupId = includeMatcher.group(1);
+						String artifactId = includeMatcher.group(2);
+						String version = includeMatcher.group(3);
+						if (!"org.apache.flink".equals(groupId)) {
+							result.put(currentShadeModule, IncludedDependency.create(groupId, artifactId, version));
+						}
+					}
+				}
+
+				if (currentDependencyCopyModule != null) {
+					Matcher copyMatcher = DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN.matcher(line);
+					if (copyMatcher.find()) {
+						String groupId = copyMatcher.group(1);
+						String artifactId = copyMatcher.group(2);
+						String version = copyMatcher.group(3);
+						if (!"org.apache.flink".equals(groupId)) {
+							result.put(currentDependencyCopyModule, IncludedDependency.create(groupId, artifactId, version));
+						}
+					}
+				}
+				if (line.contains("Replacing original artifact with shaded artifact")) {
+					currentShadeModule = null;
+				}
+				if (line.contains("Copying")) {
+					currentDependencyCopyModule = null;
+				}
+			}

Review comment:
       ```suggestion
   			});
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517211857



##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,329 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+//CHECKSTYLE.OFF: regexp|imports
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+//CHECKSTYLE.ON: regexp|imports
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), Paths.get(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:[^:]+:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	// Examples:
+	// "- org.apache.htrace:htrace-core:3.1.0-incubating"
+	// or
+	// "This project bundles "net.jcip:jcip-annotations:1.0".
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, Path root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithBundledDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithBundledDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithBundledDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<Path> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithBundledDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (Path noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithBundledDependencies, noticeFile);
+		}
+
+		return severeIssueCount;
+	}
+
+	private static int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<Path> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(Path noticeFile) {
+		Path moduleDirectory = noticeFile.getParent() // META-INF
+			.getParent() // resources
+			.getParent() // main
+			.getParent() // src
+			.getParent(); // <-- module name
+		return moduleDirectory.getFileName().toString();
+	}
+
+	private static int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, Path noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		List<String> noticeContents = Files.readAllLines(noticeFile);
+
+		if (noticeContents.isEmpty()) {
+			LOG.error("Notice file empty {}", noticeFile);
+			severeIssueCount++;
+		}
+
+		// first line must be the module name.
+		if (!noticeContents.get(0).equals(moduleName)) {
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, noticeContents.get(0));
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		for (String line : noticeContents) {
+			Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+			if (noticeDependencyMatcher.find()) {
+				String groupId = noticeDependencyMatcher.group(1);
+				String artifactId = noticeDependencyMatcher.group(2);
+				String version = noticeDependencyMatcher.group(3);
+				if (groupId == null && artifactId == null && version == null) { // "bundles" case
+					groupId = noticeDependencyMatcher.group(5);
+					artifactId = noticeDependencyMatcher.group(6);
+					version = noticeDependencyMatcher.group(7);
+				}
+				IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+				if (!declaredDependencies.add(toAdd)) {
+					LOG.error("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					severeIssueCount++;
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Collection<IncludedDependency> expectedDependencies = modulesWithShadedDependencies.get(moduleName);
+		for (IncludedDependency expectedDependency : expectedDependencies) {
+			if (!declaredDependencies.contains(expectedDependency)) {
+				LOG.error("Could not find dependency {} in NOTICE file {}", expectedDependency, noticeFile);
+				severeIssueCount++;
+			}
+		}
+
+		boolean moduleDefinesExcessDependencies = MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName);
+
+		// print all dependencies defined in NOTICE file, which were not expected
+		for (IncludedDependency declaredDependency : declaredDependencies) {
+			if (!expectedDependencies.contains(declaredDependency)) {
+				if (moduleDefinesExcessDependencies) {
+					LOG.debug(
+						"Dependency {} is mentioned in NOTICE file {}, but was not mentioned by the build output as a bundled dependency",
+						declaredDependency,
+						noticeFile);
+				} else {
+					LOG.warn(
+						"Dependency {} is mentioned in NOTICE file {}, but is not expected there",
+						declaredDependency,
+						noticeFile);
+				}
+
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static List<Path> findNoticeFiles(Path root) throws IOException {
+		return Files.walk(root)
+			.filter(file -> {
+				int nameCount = file.getNameCount();
+				return file.getName(nameCount - 3).toString().equals("resources")
+					&& file.getName(nameCount - 2).toString().equals("META-INF")
+					&& file.getName(nameCount - 1).toString().equals("NOTICE");
+			})
+			.collect(Collectors.toList());
+	}
+
+	private static Multimap<String, IncludedDependency> parseModulesFromBuildResult(File buildResult) throws IOException {
+		Multimap<String, IncludedDependency> result = ArrayListMultimap.create();
+
+		try (Stream<String> lines = Files.lines(buildResult.toPath())) {
+			//String line;
+			String currentShadeModule = null;
+			String currentDependencyCopyModule = null;
+			for (String line : (Iterable<String>) lines::iterator) {

Review comment:
       let's do it like this then (+ my comment below)
   
   ```suggestion
   			String currentShadeModule = null;
   			String currentDependencyCopyModule = null;
   			lines.forEach(line -> {
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-716665623


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit e58da69e4aeb75fddcdb08c1ae89c2bc9789902a (Mon Oct 26 16:28:09 UTC 2020)
   
   **Warnings:**
    * **2 pom.xml files were touched**: Check for build and licensing issues.
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-721756082


   I see. Thanks a lot for your patience and the explanations.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] rmetzger commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517185662



##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+//CHECKSTYLE.OFF: regexp|imports
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+//CHECKSTYLE.ON: regexp|imports
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), Paths.get(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:[^:]+:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	// Examples:
+	// "- org.apache.htrace:htrace-core:3.1.0-incubating"
+	// or
+	// "This project bundles "net.jcip:jcip-annotations:1.0".
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, Path root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithBundledDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithBundledDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithBundledDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<Path> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithBundledDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (Path noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithBundledDependencies, noticeFile);
+		}
+
+		return severeIssueCount;
+	}
+
+	private static int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<Path> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(Path noticeFile) {
+		Path moduleDirectory = noticeFile.getParent() // META-INF
+			.getParent() // resources
+			.getParent() // main
+			.getParent() // src
+			.getParent(); // <-- module name
+		return moduleDirectory.getFileName().toString();
+	}
+
+	private static int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, Path noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		List<String> noticeContents = Files.readAllLines(noticeFile);
+
+		if (noticeContents.isEmpty()) {
+			LOG.error("Notice file empty {}", noticeFile);
+			severeIssueCount++;
+		}
+
+		// first line must be the module name.
+		if (!noticeContents.get(0).equals(moduleName)) {
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, noticeContents.get(0));
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		for (String line : noticeContents) {
+			Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+			if (noticeDependencyMatcher.find()) {
+				String groupId = noticeDependencyMatcher.group(1);
+				String artifactId = noticeDependencyMatcher.group(2);
+				String version = noticeDependencyMatcher.group(3);
+				if (groupId == null && artifactId == null && version == null) { // "bundles" case
+					groupId = noticeDependencyMatcher.group(5);
+					artifactId = noticeDependencyMatcher.group(6);
+					version = noticeDependencyMatcher.group(7);
+				}
+				IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+				if (!declaredDependencies.add(toAdd)) {
+					LOG.error("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					severeIssueCount++;
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Collection<IncludedDependency> expectedDependencies = modulesWithShadedDependencies.get(moduleName);
+		for (IncludedDependency expectedDependency : expectedDependencies) {
+			if (!declaredDependencies.contains(expectedDependency)) {
+				LOG.error("Could not find dependency {} in NOTICE file {}", expectedDependency, noticeFile);
+				severeIssueCount++;
+			}
+		}
+
+		boolean moduleDefinesExcessDependencies = MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName);
+
+		// print all dependencies defined in NOTICE file, which were not expected
+		Set<IncludedDependency> excessDependencies = new HashSet<>(declaredDependencies);
+		excessDependencies.removeAll(modulesWithShadedDependencies.get(moduleName));
+		for (IncludedDependency declaredDependency : declaredDependencies) {
+			if (!expectedDependencies.contains(declaredDependency)) {
+				if (moduleDefinesExcessDependencies) {
+					LOG.debug(
+						"Dependency {} is mentioned in NOTICE file {}, but was not mentioned by the build output as a bundled dependency",
+						declaredDependency,
+						noticeFile);
+				} else {
+					LOG.warn(
+						"Dependency {} is mentioned in NOTICE file {}, but is not expected there",
+						declaredDependency,
+						noticeFile);
+				}
+
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static List<Path> findNoticeFiles(Path root) throws IOException {
+		return Files.walk(root)
+			.filter(file -> {
+				int nameCount = file.getNameCount();
+				return file.getName(nameCount - 3).toString().equals("resources")
+					&& file.getName(nameCount - 2).toString().equals("META-INF")
+					&& file.getName(nameCount - 1).toString().equals("NOTICE");
+			})
+			.collect(Collectors.toList());
+	}
+
+	private static Multimap<String, IncludedDependency> parseModulesFromBuildResult(File buildResult) throws IOException {
+		Multimap<String, IncludedDependency> result = ArrayListMultimap.create();
+
+		try (BufferedReader br = new BufferedReader(new FileReader(buildResult))) {
+			String line;
+			String currentShadeModule = null;
+			String currentDependencyCopyModule = null;
+			while ((line = br.readLine()) != null) {

Review comment:
       In this case I have to use `for (String line : (Iterable<String>) lines::iterator) {`.
   

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/LicenseChecker.java
##########
@@ -0,0 +1,333 @@
+/*
+ * 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.tools.ci.licensecheck;
+
+//CHECKSTYLE.OFF: regexp|imports
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+//CHECKSTYLE.ON: regexp|imports
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+/**
+ * Utility class checking for proper NOTICE files based on the maven build output.
+ */
+public class LicenseChecker {
+	// ---------------------------------------- Launcher ---------------------------------------- //
+
+	private static final Logger LOG = LoggerFactory.getLogger(LicenseChecker.class);
+
+	public static void main(String[] args) throws IOException {
+		if (args.length < 2) {
+			System.out.println("Usage: LicenseChecker <pathMavenBuildOutput> <pathFlinkRoot>");
+			System.exit(1);
+		}
+		LOG.warn("THIS UTILITY IS ONLY CHECKING FOR COMMON LICENSING MISTAKES. A MANUAL CHECK OF THE NOTICE FILES, DEPLOYED ARTIFACTS, ETC. IS STILL NEEDED!");
+
+		LicenseChecker checker = new LicenseChecker();
+		int severeIssueCount = checker.run(new File(args[0]), Paths.get(args[1]));
+
+		if (severeIssueCount > 0) {
+			LOG.warn("Found a total of {} severe license issues", severeIssueCount);
+
+			System.exit(1);
+		}
+		LOG.info("License check completed without severe issues.");
+	}
+
+	// ---------------------------------------- License Checker ---------------------------------------- //
+
+	private static final List<String> MODULES_SKIPPING_DEPLOYMENT = loadFromResources("modules-skipping-deployment.modulelist");
+
+	private static final List<String> MODULES_DEFINING_EXCESS_DEPENDENCIES = loadFromResources("modules-defining-excess-dependencies.modulelist");
+
+	// pattern for maven shade plugin
+	private static final Pattern SHADE_NEXT_MODULE_PATTERN = Pattern.compile(".*:shade \\((shade-flink|default)\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern SHADE_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Including ([^:]+):([^:]+):jar:([^ ]+) in the shaded jar");
+
+	// pattern for maven-dependency-plugin copyied dependencies
+	private static final Pattern DEPENDENCY_COPY_NEXT_MODULE_PATTERN = Pattern.compile(".*maven-dependency-plugin:[^:]+:copy \\([^)]+\\) @ ([^ _]+)(_[0-9.]+)? --.*");
+	private static final Pattern DEPENDENCY_COPY_INCLUDE_MODULE_PATTERN = Pattern.compile(".*Configured Artifact: ([^:]+):([^:]+):([^:]+):jar.*");
+
+	// Examples:
+	// "- org.apache.htrace:htrace-core:3.1.0-incubating"
+	// or
+	// "This project bundles "net.jcip:jcip-annotations:1.0".
+	private static final Pattern NOTICE_DEPENDENCY_PATTERN = Pattern.compile("- ([^ :]+):([^:]+):([^ ]+)($| )|.*bundles \"([^:]+):([^:]+):([^\"]+)\".*");
+
+	private int run(File buildResult, Path root) throws IOException {
+		int severeIssueCount = 0;
+		// parse included dependencies from build output
+		Multimap<String, IncludedDependency> modulesWithBundledDependencies = parseModulesFromBuildResult(buildResult);
+		LOG.info("Extracted " + modulesWithBundledDependencies.asMap().keySet().size() + " modules with a total of " + modulesWithBundledDependencies.values().size() + " dependencies");
+
+		// find modules producing a shaded-jar
+		List<Path> noticeFiles = findNoticeFiles(root);
+		LOG.info("Found {} NOTICE files to check", noticeFiles.size());
+
+		// check that all required NOTICE files exists
+		severeIssueCount += ensureRequiredNoticeFiles(modulesWithBundledDependencies, noticeFiles);
+
+		// check each NOTICE file
+		for (Path noticeFile: noticeFiles) {
+			severeIssueCount += checkNoticeFile(modulesWithBundledDependencies, noticeFile);
+		}
+
+		return severeIssueCount;
+	}
+
+	private static int ensureRequiredNoticeFiles(Multimap<String, IncludedDependency> modulesWithShadedDependencies, List<Path> noticeFiles) {
+		int severeIssueCount = 0;
+		Set<String> shadingModules = new HashSet<>(modulesWithShadedDependencies.keys());
+		shadingModules.removeAll(noticeFiles.stream().map(LicenseChecker::getModuleFromNoticeFile).collect(Collectors.toList()));
+		for (String moduleWithoutNotice : shadingModules) {
+			if (!MODULES_SKIPPING_DEPLOYMENT.contains(moduleWithoutNotice)) {
+				LOG.warn("Module {} is missing a NOTICE file. It has shaded dependencies: {}", moduleWithoutNotice, modulesWithShadedDependencies.get(moduleWithoutNotice));
+				severeIssueCount++;
+			}
+		}
+		return severeIssueCount;
+	}
+
+	private static String getModuleFromNoticeFile(Path noticeFile) {
+		Path moduleDirectory = noticeFile.getParent() // META-INF
+			.getParent() // resources
+			.getParent() // main
+			.getParent() // src
+			.getParent(); // <-- module name
+		return moduleDirectory.getFileName().toString();
+	}
+
+	private static int checkNoticeFile(Multimap<String, IncludedDependency> modulesWithShadedDependencies, Path noticeFile) throws IOException {
+		int severeIssueCount = 0;
+		String moduleName = getModuleFromNoticeFile(noticeFile);
+
+		// 1st line contains module name
+		List<String> noticeContents = Files.readAllLines(noticeFile);
+
+		if (noticeContents.isEmpty()) {
+			LOG.error("Notice file empty {}", noticeFile);
+			severeIssueCount++;
+		}
+
+		// first line must be the module name.
+		if (!noticeContents.get(0).equals(moduleName)) {
+			LOG.warn("Expected first file of notice file to start with module name. moduleName={}, firstLine={}", moduleName, noticeContents.get(0));
+		}
+
+		// collect all declared dependencies from NOTICE file
+		Set<IncludedDependency> declaredDependencies = new HashSet<>();
+		for (String line : noticeContents) {
+			Matcher noticeDependencyMatcher = NOTICE_DEPENDENCY_PATTERN.matcher(line);
+			if (noticeDependencyMatcher.find()) {
+				String groupId = noticeDependencyMatcher.group(1);
+				String artifactId = noticeDependencyMatcher.group(2);
+				String version = noticeDependencyMatcher.group(3);
+				if (groupId == null && artifactId == null && version == null) { // "bundles" case
+					groupId = noticeDependencyMatcher.group(5);
+					artifactId = noticeDependencyMatcher.group(6);
+					version = noticeDependencyMatcher.group(7);
+				}
+				IncludedDependency toAdd = IncludedDependency.create(groupId, artifactId, version);
+				if (!declaredDependencies.add(toAdd)) {
+					LOG.error("Dependency {} has been declared twice in module {}", toAdd, moduleName);
+					severeIssueCount++;
+				}
+			}
+		}
+		// print all dependencies missing from NOTICE file
+		Collection<IncludedDependency> expectedDependencies = modulesWithShadedDependencies.get(moduleName);
+		for (IncludedDependency expectedDependency : expectedDependencies) {
+			if (!declaredDependencies.contains(expectedDependency)) {
+				LOG.error("Could not find dependency {} in NOTICE file {}", expectedDependency, noticeFile);
+				severeIssueCount++;
+			}
+		}
+
+		boolean moduleDefinesExcessDependencies = MODULES_DEFINING_EXCESS_DEPENDENCIES.contains(moduleName);
+
+		// print all dependencies defined in NOTICE file, which were not expected
+		Set<IncludedDependency> excessDependencies = new HashSet<>(declaredDependencies);
+		excessDependencies.removeAll(modulesWithShadedDependencies.get(moduleName));
+		for (IncludedDependency declaredDependency : declaredDependencies) {
+			if (!expectedDependencies.contains(declaredDependency)) {
+				if (moduleDefinesExcessDependencies) {
+					LOG.debug(
+						"Dependency {} is mentioned in NOTICE file {}, but was not mentioned by the build output as a bundled dependency",
+						declaredDependency,
+						noticeFile);
+				} else {
+					LOG.warn(
+						"Dependency {} is mentioned in NOTICE file {}, but is not expected there",
+						declaredDependency,
+						noticeFile);
+				}
+
+			}
+		}
+
+		return severeIssueCount;
+	}
+
+	private static List<Path> findNoticeFiles(Path root) throws IOException {
+		return Files.walk(root)
+			.filter(file -> {
+				int nameCount = file.getNameCount();
+				return file.getName(nameCount - 3).toString().equals("resources")
+					&& file.getName(nameCount - 2).toString().equals("META-INF")
+					&& file.getName(nameCount - 1).toString().equals("NOTICE");
+			})
+			.collect(Collectors.toList());
+	}
+
+	private static Multimap<String, IncludedDependency> parseModulesFromBuildResult(File buildResult) throws IOException {
+		Multimap<String, IncludedDependency> result = ArrayListMultimap.create();
+
+		try (BufferedReader br = new BufferedReader(new FileReader(buildResult))) {
+			String line;
+			String currentShadeModule = null;
+			String currentDependencyCopyModule = null;
+			while ((line = br.readLine()) != null) {

Review comment:
       I have applied this suggestion anyways




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol edited a comment on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol edited a comment on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-718130365


   There is no benefit in having the generated files in git, it could just be step of the release process.
   
   On CI we would compile Flink and pipe the build output into a file, then run the generator with this file as input to ensure that the NOTICE files can be generated.
   
   > I've quickly looked at building our own shade resource transformer
   
   FYI: Last time I tried it wasn't possible to write custom transformers (it _looks_ like you can, but I always had classpath issues), and even if you could it would interfere with other transformers touching the notice file (like the appender), because a file can only be touched by at most 1 transformer.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13796:
URL: https://github.com/apache/flink/pull/13796#discussion_r517232162



##########
File path: flink-connectors/flink-sql-connector-hbase-1.4/pom.xml
##########
@@ -87,6 +87,7 @@ under the License.
 									<exclude>org.apache.hbase:hbase-metrics*</exclude>
 									<exclude>org.apache.hbase:hbase-server*</exclude>
 									<exclude>org.apache.hbase:hbase-hadoop*-compat</exclude>
+									<exclude>org.apache.hbase:hbase-common:jar:tests</exclude>

Review comment:
       What about declaring it as a test dependency? I checked the hbase-procedure module, and it doesn't actually seem to require it outside of tests.
   ```
   <dependency>
   	<groupId>org.apache.hbase</groupId>
   	<artifactId>hbase-common</artifactId>
   	<version>${hbase.version}</version>
   	<classifier>tests</classifier>
   	<scope>test</scope>
   </dependency>
   ```
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on pull request #13796: [FLINK-19810][CI] Automatically run a basic NOTICE file check on CI

Posted by GitBox <gi...@apache.org>.
zentol commented on pull request #13796:
URL: https://github.com/apache/flink/pull/13796#issuecomment-717821843


   I'm still not a fan of automating the verification; people _will_ see the tool as the source of truth (which it isn't, as you pointed out), and any attempt at automating the generation can now be shut down with "well we have a tool to check the correctness, so let's not spend time on it".
   
   That said, it is obvious that it does find issues; particularly subtle ones that can _easily_ be missed during a manual review, `com` vs `org` is subtle AF and not really the focus of such a review (as it is more concerned with anything being listed at all and the version being correct).
   
   So I am, begrudgingly, fine with it.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org