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/12/03 09:40:21 UTC

[GitHub] [flink] rmetzger opened a new pull request #14296: [FLINK-20455] Add check for jar file contents

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


   
   ## What is the purpose of the change
   
   During the release validation, we noticed several jar files containing LICENSE files in their root.
   
   This change adds some basic checks to validate the jar files.
   
   NOTICE: The check currently fails because the examples lack LICENSE files.
   
   ## Brief change log
   
   - Split license checker into notice and jar checker
   - add jar checker
   - fix flink-table related issues
   
   
   
   


----------------------------------------------------------------
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 #14296: [FLINK-20455] Add check for jar file contents

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



##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/JarFileChecker.java
##########
@@ -0,0 +1,97 @@
+/*
+ * 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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
+
+/**
+ * Checks the Jar files created by the build process
+ */
+public class JarFileChecker {
+	private static final Logger LOG = LoggerFactory.getLogger(JarFileChecker.class);
+
+	public int run(Path path) throws IOException {
+		List<Path> files = getBuildJars(path);
+
+		LOG.info("considering jar files " + files);
+
+		int severeIssues = 0;
+		for (Path file: files) {
+			severeIssues += checkJar(file);
+		}
+
+		return severeIssues;
+	}
+
+	private int checkJar(Path file) throws IOException {
+		int severeIssues = 0;
+		boolean metaInfNoticeSeen = false;
+		boolean metaInfLicenseSeen = false;
+
+		try (ZipInputStream zis = new ZipInputStream(new FileInputStream(file.toFile()))) {
+			ZipEntry ze;
+			while ((ze = zis.getNextEntry()) != null) {
+				if (ze.getName().equals("LICENSE")) {
+					LOG.error("Jar file {} contains a LICENSE file in the root folder", file);
+					severeIssues++;
+				}
+				if (ze.getName().equals("META-INF/NOTICE")) {
+					metaInfNoticeSeen = true;
+				}
+				if (ze.getName().equals("META-INF/LICENSE")) {

Review comment:
       we should also reject META-INF/LICENSE.txt

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/JarFileChecker.java
##########
@@ -0,0 +1,97 @@
+/*
+ * 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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
+
+/**
+ * Checks the Jar files created by the build process
+ */
+public class JarFileChecker {
+	private static final Logger LOG = LoggerFactory.getLogger(JarFileChecker.class);
+
+	public int run(Path path) throws IOException {
+		List<Path> files = getBuildJars(path);
+
+		LOG.info("considering jar files " + files);
+
+		int severeIssues = 0;
+		for (Path file: files) {
+			severeIssues += checkJar(file);
+		}
+
+		return severeIssues;
+	}
+
+	private int checkJar(Path file) throws IOException {
+		int severeIssues = 0;
+		boolean metaInfNoticeSeen = false;
+		boolean metaInfLicenseSeen = false;
+
+		try (ZipInputStream zis = new ZipInputStream(new FileInputStream(file.toFile()))) {
+			ZipEntry ze;
+			while ((ze = zis.getNextEntry()) != null) {
+				if (ze.getName().equals("LICENSE")) {
+					LOG.error("Jar file {} contains a LICENSE file in the root folder", file);

Review comment:
       Let's explicitly give instructions on what to do here; validate whether the license is required, manually adding it to our licensing setup, and adding an exclusion to the shade plugin.

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/JarFileChecker.java
##########
@@ -0,0 +1,97 @@
+/*
+ * 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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
+
+/**
+ * Checks the Jar files created by the build process
+ */
+public class JarFileChecker {
+	private static final Logger LOG = LoggerFactory.getLogger(JarFileChecker.class);
+
+	public int run(Path path) throws IOException {
+		List<Path> files = getBuildJars(path);
+
+		LOG.info("considering jar files " + files);
+
+		int severeIssues = 0;
+		for (Path file: files) {
+			severeIssues += checkJar(file);
+		}
+
+		return severeIssues;
+	}
+
+	private int checkJar(Path file) throws IOException {
+		int severeIssues = 0;
+		boolean metaInfNoticeSeen = false;
+		boolean metaInfLicenseSeen = false;
+
+		try (ZipInputStream zis = new ZipInputStream(new FileInputStream(file.toFile()))) {
+			ZipEntry ze;
+			while ((ze = zis.getNextEntry()) != null) {
+				if (ze.getName().equals("LICENSE")) {

Review comment:
       maybe instead check that no root file contains the word "license"?




----------------------------------------------------------------
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 #14296: [FLINK-20455] Add check for jar file contents

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


   I pushed a new version, with more checks, which addresses all issues found by the extended tool.


----------------------------------------------------------------
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 #14296: [FLINK-20455] Add check for jar file contents

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



##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/JarFileChecker.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 org.apache.logging.log4j.core.util.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
+
+/**
+ * Checks the Jar files created by the build process.
+ */
+public class JarFileChecker {
+	private static final Logger LOG = LoggerFactory.getLogger(JarFileChecker.class);
+
+	public int run(Path path) throws IOException {
+		List<Path> files = getBuildJars(path);
+		LOG.info("Checking directory {} with a total of {} jar files.", path, files.size());
+
+		int severeIssues = 0;
+		for (Path file: files) {
+			severeIssues += checkJar(file);
+		}
+
+		return severeIssues;
+	}
+
+	private int checkJar(Path file) throws IOException {
+		boolean metaInfNoticeSeen = false;
+		boolean metaInfLicenseSeen = false;
+		int classFiles = 0;
+
+		List<String> errors = new ArrayList<>();
+		try (ZipInputStream zis = new ZipInputStream(new FileInputStream(file.toFile()))) {
+			ZipEntry ze;
+			while ((ze = zis.getNextEntry()) != null) {
+				final String name = ze.getName();
+				/**
+				 * There must be a LICENSE file and NOTICE file in the META-INF directory.
+				 * LICENSE or NOTICE files found outside of the META-INF directory are most likely shading mistakes (we are including the files from other dependencies, thus providing an invalid LICENSE file)
+				 *
+				 * In such a case, we recommend updating the shading exclusions, and adding the license file to META-INF/licenses.
+				 */
+				if (!name.contains("META-INF") // license files in META-INF are expected
+					&& !name.contains(".class") // some class files contain LICENSE in their name
+					&& !name.contains(".ftl") // a false positive in flink-python
+					&& !name.contains("web/3rdpartylicenses.txt") // a false positive in flink-runtime-web
+					&& !name.endsWith("/") // ignore directories, e.g. "license/"
+					&& name.toUpperCase().contains("LICENSE")) {
+					errors.add("Jar file " + file + " contains a LICENSE file in an unexpected location: " + name);
+				}
+				if (!name.contains("META-INF") && !name.contains(".class") && name.toUpperCase().contains("NOTICE")) {
+					errors.add("Jar file " + file + " contains a NOTICE file in an unexpected location: " + name);
+				}
+
+				if (name.equals("META-INF/NOTICE")) {
+					String noticeContents =  IOUtils.toString(new InputStreamReader(zis));
+					if (!noticeContents.toLowerCase().contains("flink") || !noticeContents.contains("The Apache Software Foundation")) {
+						errors.add("The notice file in " + file + ":META-INF/NOTICE does not contain the expected entries");
+					}
+					metaInfNoticeSeen = true;
+				}
+				if (name.equals("META-INF/LICENSE")) {
+					String licenseContents =  IOUtils.toString(new InputStreamReader(zis));
+					if (!licenseContents.contains("Apache License") || !licenseContents.contains("Version 2.0, January 2004")) {
+						errors.add("The notice file in " + file + ":META-INF/LICENSE does not contain the expected entries");
+					}
+					metaInfLicenseSeen = true;
+				}
+				if (name.endsWith(".class")) {
+					classFiles++;
+				}
+				zis.closeEntry();
+			}
+		}
+
+		// Empty test jars (with no class files) are skipped
+		if (classFiles == 0 && file.toString().endsWith("-tests.jar")) {
+			return 0;
+		}
+		for (String error: errors) {
+			LOG.warn(error);
+		}
+		int severeIssues = errors.size();
+
+		if (!metaInfLicenseSeen) {
+			LOG.warn("Missing META-INF/LICENSE in {}", file);
+			severeIssues++;
+		}
+
+		if (!metaInfNoticeSeen) {
+			LOG.warn("Missing META-INF/NOTICE in {}", file);
+			severeIssues++;
+		}
+		return severeIssues;

Review comment:
       ```suggestion
   		int numSevereIssues = 0;
   		try (final FileSystem fileSystem = FileSystems.newFileSystem(new URI("jar:file", uri.getHost(), uri.getPath(), uri.getFragment()), Collections.emptyMap())) {
   
   			// test jars (with no class files) are ignored
   			if (file.getFileName().toString().endsWith("-tests.jar")) {
   				for (Path root : fileSystem.getRootDirectories()) {
   					try (Stream<Path> files = Files.walk(root)) {
   						long numClassFiles = files
   								.filter(path -> path.getFileName().toString().endsWith(".class"))
   								.count();
   
   						if (numClassFiles == 0) {
   							return 0;
   						}
   					}
   				}
   			}
   
   			// check notice file existence and contents
   			final Path noticeFile = fileSystem.getPath("META-INF", "NOTICE");
   			if (Files.exists(noticeFile)) {
   				final String noticeFileContents = Files.readString(noticeFile);
   				if (!noticeFileContents.toLowerCase().contains("flink") || !noticeFileContents.contains("The Apache Software Foundation")) {
   					LOG.error("The notify file in {} does not contain the expected entries.", file);
   					numSevereIssues++;
   				}
   			} else {
   				LOG.error("Missing META-INF/NOTICE in {}", file);
   				numSevereIssues++;
   			}
   
   			// check license file existence and contents
   			final Path licenseFile = fileSystem.getPath("META-INF", "LICENSE");
   			if (Files.exists(licenseFile)) {
   				final String licenseFileContents = Files.readString(licenseFile);
   				if (!licenseFileContents.contains("Apache License") || !licenseFileContents.contains("Version 2.0, January 2004")) {
   					LOG.error("The license file in {} does not contain the expected entries.", file);
   					numSevereIssues++;
   				}
   			} else {
   				LOG.error("Missing META-INF/NOTICE in {}", file);
   				numSevereIssues++;
   			}
   
   			for (Path root : fileSystem.getRootDirectories()) {
   				try (Stream<Path> files = Files.walk(root)) {
   					/*
   					 * LICENSE or NOTICE files found outside of the META-INF directory are most likely shading mistakes (we are including the files from other dependencies, thus providing an invalid LICENSE file)
   					 *
   					 * <p>In such a case, we recommend updating the shading exclusions, and adding the license file to META-INF/licenses.
   					 */
   					final List<String> filesWithIssues = files
   						.filter(path -> !path.equals(root))
   						.filter(path -> path.getFileName().toString().toLowerCase().contains("license"))
   						.filter(path -> !Files.isDirectory(path)) // ignore directories, e.g. "license/"
   						.filter(path -> !path.getFileName().toString().endsWith(".class")) // some class files contain LICENSE in their name
   						.filter(path -> !path.getFileName().toString().endsWith(".ftl")) // a false positive in flink-python
   						.map(Path::toString)
   						.filter(path -> !path.contains("META-INF")) // license files in META-INF are expected
   						.filter(path -> !path.endsWith("web/3rdpartylicenses.txt")) // a false positive in flink-runtime-web
   						.collect(Collectors.toList());
   
   					numSevereIssues += filesWithIssues.size();
   					for (String fileWithIssue : filesWithIssues) {
   						LOG.error("Jar file {} contains a LICENSE file in an unexpected location: {}", file, fileWithIssue);
   					}
   				}
   			}
   		}
   
   		return numSevereIssues;
   ```

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/JarFileChecker.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 org.apache.logging.log4j.core.util.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
+
+/**
+ * Checks the Jar files created by the build process.
+ */
+public class JarFileChecker {
+	private static final Logger LOG = LoggerFactory.getLogger(JarFileChecker.class);
+
+	public int run(Path path) throws IOException {
+		List<Path> files = getBuildJars(path);
+		LOG.info("Checking directory {} with a total of {} jar files.", path, files.size());
+
+		int severeIssues = 0;
+		for (Path file: files) {
+			severeIssues += checkJar(file);
+		}
+
+		return severeIssues;
+	}
+
+	private int checkJar(Path file) throws IOException {
+		boolean metaInfNoticeSeen = false;
+		boolean metaInfLicenseSeen = false;
+		int classFiles = 0;
+
+		List<String> errors = new ArrayList<>();
+		try (ZipInputStream zis = new ZipInputStream(new FileInputStream(file.toFile()))) {
+			ZipEntry ze;
+			while ((ze = zis.getNextEntry()) != null) {
+				final String name = ze.getName();
+				/**
+				 * There must be a LICENSE file and NOTICE file in the META-INF directory.
+				 * LICENSE or NOTICE files found outside of the META-INF directory are most likely shading mistakes (we are including the files from other dependencies, thus providing an invalid LICENSE file)
+				 *
+				 * In such a case, we recommend updating the shading exclusions, and adding the license file to META-INF/licenses.
+				 */
+				if (!name.contains("META-INF") // license files in META-INF are expected
+					&& !name.contains(".class") // some class files contain LICENSE in their name
+					&& !name.contains(".ftl") // a false positive in flink-python
+					&& !name.contains("web/3rdpartylicenses.txt") // a false positive in flink-runtime-web
+					&& !name.endsWith("/") // ignore directories, e.g. "license/"
+					&& name.toUpperCase().contains("LICENSE")) {
+					errors.add("Jar file " + file + " contains a LICENSE file in an unexpected location: " + name);
+				}
+				if (!name.contains("META-INF") && !name.contains(".class") && name.toUpperCase().contains("NOTICE")) {
+					errors.add("Jar file " + file + " contains a NOTICE file in an unexpected location: " + name);
+				}
+
+				if (name.equals("META-INF/NOTICE")) {
+					String noticeContents =  IOUtils.toString(new InputStreamReader(zis));
+					if (!noticeContents.toLowerCase().contains("flink") || !noticeContents.contains("The Apache Software Foundation")) {
+						errors.add("The notice file in " + file + ":META-INF/NOTICE does not contain the expected entries");
+					}
+					metaInfNoticeSeen = true;
+				}
+				if (name.equals("META-INF/LICENSE")) {
+					String licenseContents =  IOUtils.toString(new InputStreamReader(zis));
+					if (!licenseContents.contains("Apache License") || !licenseContents.contains("Version 2.0, January 2004")) {
+						errors.add("The notice file in " + file + ":META-INF/LICENSE does not contain the expected entries");
+					}
+					metaInfLicenseSeen = true;
+				}
+				if (name.endsWith(".class")) {
+					classFiles++;
+				}
+				zis.closeEntry();
+			}
+		}
+
+		// Empty test jars (with no class files) are skipped
+		if (classFiles == 0 && file.toString().endsWith("-tests.jar")) {
+			return 0;
+		}
+		for (String error: errors) {
+			LOG.warn(error);
+		}
+		int severeIssues = errors.size();
+
+		if (!metaInfLicenseSeen) {
+			LOG.warn("Missing META-INF/LICENSE in {}", file);
+			severeIssues++;
+		}
+
+		if (!metaInfNoticeSeen) {
+			LOG.warn("Missing META-INF/NOTICE in {}", file);
+			severeIssues++;
+		}
+		return severeIssues;

Review comment:
       ```suggestion
   		int numSevereIssues = 0;
   		try (final FileSystem fileSystem = FileSystems.newFileSystem(new URI("jar:file", uri.getHost(), uri.getPath(), uri.getFragment()), Collections.emptyMap())) {
   
   			// test jars (with no class files) are ignored
   			if (file.getFileName().toString().endsWith("-tests.jar")) {
   				for (Path root : fileSystem.getRootDirectories()) {
   					try (Stream<Path> files = Files.walk(root)) {
   						long numClassFiles = files
   							.filter(path -> path.getFileName().toString().endsWith(".class"))
   							.count();
   
   						if (numClassFiles == 0) {
   							return 0;
   						}
   					}
   				}
   			}
   
   			// check notice file existence and contents
   			final Path noticeFile = fileSystem.getPath("META-INF", "NOTICE");
   			if (Files.exists(noticeFile)) {
   				final String noticeFileContents = Files.readString(noticeFile);
   				if (!noticeFileContents.toLowerCase().contains("flink") || !noticeFileContents.contains("The Apache Software Foundation")) {
   					LOG.error("The notify file in {} does not contain the expected entries.", file);
   					numSevereIssues++;
   				}
   			} else {
   				LOG.error("Missing META-INF/NOTICE in {}", file);
   				numSevereIssues++;
   			}
   
   			// check license file existence and contents
   			final Path licenseFile = fileSystem.getPath("META-INF", "LICENSE");
   			if (Files.exists(licenseFile)) {
   				final String licenseFileContents = Files.readString(licenseFile);
   				if (!licenseFileContents.contains("Apache License") || !licenseFileContents.contains("Version 2.0, January 2004")) {
   					LOG.error("The license file in {} does not contain the expected entries.", file);
   					numSevereIssues++;
   				}
   			} else {
   				LOG.error("Missing META-INF/NOTICE in {}", file);
   				numSevereIssues++;
   			}
   
   			for (Path root : fileSystem.getRootDirectories()) {
   				try (Stream<Path> files = Files.walk(root)) {
   					/*
   					 * LICENSE or NOTICE files found outside of the META-INF directory are most likely shading mistakes (we are including the files from other dependencies, thus providing an invalid LICENSE file)
   					 *
   					 * <p>In such a case, we recommend updating the shading exclusions, and adding the license file to META-INF/licenses.
   					 */
   					final List<String> filesWithIssues = files
   						.filter(path -> !path.equals(root))
   						.filter(path -> path.getFileName().toString().toLowerCase().contains("license"))
   						.filter(path -> !Files.isDirectory(path)) // ignore directories, e.g. "license/"
   						.filter(path -> !path.getFileName().toString().endsWith(".class")) // some class files contain LICENSE in their name
   						.filter(path -> !path.getFileName().toString().endsWith(".ftl")) // a false positive in flink-python
   						.map(Path::toString)
   						.filter(path -> !path.contains("META-INF")) // license files in META-INF are expected
   						.filter(path -> !path.endsWith("web/3rdpartylicenses.txt")) // a false positive in flink-runtime-web
   						.collect(Collectors.toList());
   
   					numSevereIssues += filesWithIssues.size();
   					for (String fileWithIssue : filesWithIssues) {
   						LOG.error("Jar file {} contains a LICENSE file in an unexpected location: {}", file, fileWithIssue);
   					}
   				}
   			}
   		}
   
   		return numSevereIssues;
   ```




----------------------------------------------------------------
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 #14296: [FLINK-20455] Add check for jar file contents

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


   Manually merged.


----------------------------------------------------------------
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 #14296: [FLINK-20455] Add check for jar file contents

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1a102565515cc8116fa304d69dd1746739f8aa3e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1a102565515cc8116fa304d69dd1746739f8aa3e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1a102565515cc8116fa304d69dd1746739f8aa3e 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 #14296: [FLINK-20455] Add check for jar file contents

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1a102565515cc8116fa304d69dd1746739f8aa3e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10477",
       "triggerID" : "1a102565515cc8116fa304d69dd1746739f8aa3e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1a102565515cc8116fa304d69dd1746739f8aa3e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10477) 
   
   <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 #14296: [FLINK-20455] Add check for jar file contents

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1a102565515cc8116fa304d69dd1746739f8aa3e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10477",
       "triggerID" : "1a102565515cc8116fa304d69dd1746739f8aa3e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0aa70d75d3fb004a910eabfa43d3f7a37bb2f56c",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10493",
       "triggerID" : "0aa70d75d3fb004a910eabfa43d3f7a37bb2f56c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1a102565515cc8116fa304d69dd1746739f8aa3e Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10477) 
   * 0aa70d75d3fb004a910eabfa43d3f7a37bb2f56c Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10493) 
   
   <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 closed pull request #14296: [FLINK-20455] Add check for jar file contents

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


   


----------------------------------------------------------------
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 #14296: [FLINK-20455] Add check for jar file contents

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1a102565515cc8116fa304d69dd1746739f8aa3e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10477",
       "triggerID" : "1a102565515cc8116fa304d69dd1746739f8aa3e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0aa70d75d3fb004a910eabfa43d3f7a37bb2f56c",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10493",
       "triggerID" : "0aa70d75d3fb004a910eabfa43d3f7a37bb2f56c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0aa70d75d3fb004a910eabfa43d3f7a37bb2f56c Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10493) 
   
   <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 #14296: [FLINK-20455] Add check for jar file contents

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1a102565515cc8116fa304d69dd1746739f8aa3e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10477",
       "triggerID" : "1a102565515cc8116fa304d69dd1746739f8aa3e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1a102565515cc8116fa304d69dd1746739f8aa3e Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10477) 
   
   <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 #14296: [FLINK-20455] Add check for jar file contents

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



##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/JarFileChecker.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 org.apache.logging.log4j.core.util.IOUtils;

Review comment:
       oh, damn, that was not intentional ;)
   




----------------------------------------------------------------
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 #14296: [FLINK-20455] Add check for jar file contents

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



##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/JarFileChecker.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 org.apache.logging.log4j.core.util.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
+
+/**
+ * Checks the Jar files created by the build process.
+ */
+public class JarFileChecker {
+	private static final Logger LOG = LoggerFactory.getLogger(JarFileChecker.class);
+
+	public int run(Path path) throws IOException {
+		List<Path> files = getBuildJars(path);
+		LOG.info("Checking directory {} with a total of {} jar files.", path, files.size());
+
+		int severeIssues = 0;
+		for (Path file: files) {
+			severeIssues += checkJar(file);
+		}
+
+		return severeIssues;
+	}
+
+	private int checkJar(Path file) throws IOException {
+		boolean metaInfNoticeSeen = false;
+		boolean metaInfLicenseSeen = false;
+		int classFiles = 0;
+
+		List<String> errors = new ArrayList<>();
+		try (ZipInputStream zis = new ZipInputStream(new FileInputStream(file.toFile()))) {
+			ZipEntry ze;
+			while ((ze = zis.getNextEntry()) != null) {
+				final String name = ze.getName();
+				/**
+				 * There must be a LICENSE file and NOTICE file in the META-INF directory.
+				 * LICENSE or NOTICE files found outside of the META-INF directory are most likely shading mistakes (we are including the files from other dependencies, thus providing an invalid LICENSE file)
+				 *
+				 * In such a case, we recommend updating the shading exclusions, and adding the license file to META-INF/licenses.
+				 */
+				if (!name.contains("META-INF") // license files in META-INF are expected
+					&& !name.contains(".class") // some class files contain LICENSE in their name
+					&& !name.contains(".ftl") // a false positive in flink-python
+					&& !name.contains("web/3rdpartylicenses.txt") // a false positive in flink-runtime-web
+					&& !name.endsWith("/") // ignore directories, e.g. "license/"
+					&& name.toUpperCase().contains("LICENSE")) {
+					errors.add("Jar file " + file + " contains a LICENSE file in an unexpected location: " + name);
+				}
+				if (!name.contains("META-INF") && !name.contains(".class") && name.toUpperCase().contains("NOTICE")) {
+					errors.add("Jar file " + file + " contains a NOTICE file in an unexpected location: " + name);
+				}
+
+				if (name.equals("META-INF/NOTICE")) {
+					String noticeContents =  IOUtils.toString(new InputStreamReader(zis));
+					if (!noticeContents.toLowerCase().contains("flink") || !noticeContents.contains("The Apache Software Foundation")) {
+						errors.add("The notice file in " + file + ":META-INF/NOTICE does not contain the expected entries");
+					}
+					metaInfNoticeSeen = true;
+				}
+				if (name.equals("META-INF/LICENSE")) {
+					String licenseContents =  IOUtils.toString(new InputStreamReader(zis));
+					if (!licenseContents.contains("Apache License") || !licenseContents.contains("Version 2.0, January 2004")) {
+						errors.add("The notice file in " + file + ":META-INF/LICENSE does not contain the expected entries");
+					}
+					metaInfLicenseSeen = true;
+				}
+				if (name.endsWith(".class")) {
+					classFiles++;
+				}
+				zis.closeEntry();
+			}
+		}
+
+		// Empty test jars (with no class files) are skipped
+		if (classFiles == 0 && file.toString().endsWith("-tests.jar")) {
+			return 0;
+		}
+		for (String error: errors) {
+			LOG.warn(error);
+		}
+		int severeIssues = errors.size();
+
+		if (!metaInfLicenseSeen) {
+			LOG.warn("Missing META-INF/LICENSE in {}", file);
+			severeIssues++;
+		}
+
+		if (!metaInfNoticeSeen) {
+			LOG.warn("Missing META-INF/NOTICE in {}", file);
+			severeIssues++;
+		}
+		return severeIssues;

Review comment:
       I seem to like these overly complex loops, which you dislike ;) (understandably)
   Your proposal seems to require Java 11, and it fails with a NPE. But since you are taking over, I'll leave it to you to adjust 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] flinkbot commented on pull request #14296: [FLINK-20455] Add check for jar file contents

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


   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 1a102565515cc8116fa304d69dd1746739f8aa3e (Thu Dec 03 09:42:08 UTC 2020)
   
   **Warnings:**
    * **1 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!
    * **This pull request references an unassigned [Jira ticket](https://issues.apache.org/jira/browse/FLINK-20455).** According to the [code contribution guide](https://flink.apache.org/contributing/contribute-code.html), tickets need to be assigned before starting with the implementation work.
   
   
   <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] flinkbot edited a comment on pull request #14296: [FLINK-20455] Add check for jar file contents

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


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "1a102565515cc8116fa304d69dd1746739f8aa3e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10477",
       "triggerID" : "1a102565515cc8116fa304d69dd1746739f8aa3e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0aa70d75d3fb004a910eabfa43d3f7a37bb2f56c",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0aa70d75d3fb004a910eabfa43d3f7a37bb2f56c",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 1a102565515cc8116fa304d69dd1746739f8aa3e Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10477) 
   * 0aa70d75d3fb004a910eabfa43d3f7a37bb2f56c 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 #14296: [FLINK-20455] Add check for jar file contents

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


   How about we merge the licensing issues fixes right away (because they do look correct to me), and I'll take care of writing the test tomorrow?


----------------------------------------------------------------
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 #14296: [FLINK-20455] Add check for jar file contents

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



##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/JarFileChecker.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 org.apache.logging.log4j.core.util.IOUtils;

Review comment:
       please don't add hard dependencies on log4j

##########
File path: tools/ci/java-ci-tools/src/main/java/org/apache/flink/tools/ci/licensecheck/JarFileChecker.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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 org.apache.logging.log4j.core.util.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
+
+/**
+ * Checks the Jar files created by the build process.
+ */
+public class JarFileChecker {
+	private static final Logger LOG = LoggerFactory.getLogger(JarFileChecker.class);
+
+	public int run(Path path) throws IOException {
+		List<Path> files = getBuildJars(path);
+		LOG.info("Checking directory {} with a total of {} jar files.", path, files.size());
+
+		int severeIssues = 0;
+		for (Path file: files) {
+			severeIssues += checkJar(file);
+		}
+
+		return severeIssues;
+	}
+
+	private int checkJar(Path file) throws IOException {
+		boolean metaInfNoticeSeen = false;
+		boolean metaInfLicenseSeen = false;
+		int classFiles = 0;
+
+		List<String> errors = new ArrayList<>();
+		try (ZipInputStream zis = new ZipInputStream(new FileInputStream(file.toFile()))) {
+			ZipEntry ze;
+			while ((ze = zis.getNextEntry()) != null) {
+				final String name = ze.getName();
+				/**
+				 * There must be a LICENSE file and NOTICE file in the META-INF directory.
+				 * LICENSE or NOTICE files found outside of the META-INF directory are most likely shading mistakes (we are including the files from other dependencies, thus providing an invalid LICENSE file)
+				 *
+				 * In such a case, we recommend updating the shading exclusions, and adding the license file to META-INF/licenses.
+				 */
+				if (!name.contains("META-INF") // license files in META-INF are expected
+					&& !name.contains(".class") // some class files contain LICENSE in their name
+					&& !name.contains(".ftl") // a false positive in flink-python
+					&& !name.contains("web/3rdpartylicenses.txt") // a false positive in flink-runtime-web
+					&& !name.endsWith("/") // ignore directories, e.g. "license/"
+					&& name.toUpperCase().contains("LICENSE")) {
+					errors.add("Jar file " + file + " contains a LICENSE file in an unexpected location: " + name);
+				}
+				if (!name.contains("META-INF") && !name.contains(".class") && name.toUpperCase().contains("NOTICE")) {
+					errors.add("Jar file " + file + " contains a NOTICE file in an unexpected location: " + name);
+				}
+
+				if (name.equals("META-INF/NOTICE")) {
+					String noticeContents =  IOUtils.toString(new InputStreamReader(zis));
+					if (!noticeContents.toLowerCase().contains("flink") || !noticeContents.contains("The Apache Software Foundation")) {
+						errors.add("The notice file in " + file + ":META-INF/NOTICE does not contain the expected entries");
+					}
+					metaInfNoticeSeen = true;
+				}
+				if (name.equals("META-INF/LICENSE")) {
+					String licenseContents =  IOUtils.toString(new InputStreamReader(zis));
+					if (!licenseContents.contains("Apache License") || !licenseContents.contains("Version 2.0, January 2004")) {
+						errors.add("The notice file in " + file + ":META-INF/LICENSE does not contain the expected entries");
+					}
+					metaInfLicenseSeen = true;
+				}
+				if (name.endsWith(".class")) {
+					classFiles++;
+				}
+				zis.closeEntry();
+			}
+		}
+
+		// Empty test jars (with no class files) are skipped
+		if (classFiles == 0 && file.toString().endsWith("-tests.jar")) {
+			return 0;
+		}
+		for (String error: errors) {
+			LOG.warn(error);
+		}
+		int severeIssues = errors.size();
+
+		if (!metaInfLicenseSeen) {
+			LOG.warn("Missing META-INF/LICENSE in {}", file);
+			severeIssues++;
+		}
+
+		if (!metaInfNoticeSeen) {
+			LOG.warn("Missing META-INF/NOTICE in {}", file);
+			severeIssues++;
+		}
+		return severeIssues;

Review comment:
       ```suggestion
   		int numSevereIssues = 0;
   		try (final FileSystem fileSystem = FileSystems.newFileSystem(new URI("jar:file", uri.getHost(), uri.getPath(), uri.getFragment()), Collections.emptyMap())) {
   
   			// test jars (with no class files) are ignored
   			if (file.getFileName().toString().endsWith("-tests.jar")) {
   				for (Path root : fileSystem.getRootDirectories()) {
   					try (Stream<Path> files = Files.walk(root)) {
   						long numClassFiles = files
   								.filter(path -> path.getFileName().toString().endsWith(".class"))
   								.count();
   
   						if (numClassFiles == 0) {
   							return 0;
   						}
   					}
   				}
   			}
   
   			// check notice file existence and contents
   			final Path noticeFile = fileSystem.getPath("META-INF", "NOTICE");
   			if (Files.exists(noticeFile)) {
   				final String noticeFileContents = Files.readString(noticeFile);
   				if (!noticeFileContents.toLowerCase().contains("flink") || !noticeFileContents.contains("The Apache Software Foundation")) {
   					LOG.error("The notify file in {} does not contain the expected entries.", file);
   					numSevereIssues++;
   				}
   			} else {
   				LOG.error("Missing META-INF/NOTICE in {}", file);
   				numSevereIssues++;
   			}
   
   			// check license file existence and contents
   			final Path licenseFile = fileSystem.getPath("META-INF", "LICENSE");
   			if (Files.exists(licenseFile)) {
   				final String licenseFileContents = Files.readString(licenseFile);
   				if (!licenseFileContents.contains("Apache License") || !licenseFileContents.contains("Version 2.0, January 2004")) {
   					LOG.error("The license file in {} does not contain the expected entries.", file);
   					numSevereIssues++;
   				}
   			} else {
   				LOG.error("Missing META-INF/NOTICE in {}", file);
   				numSevereIssues++;
   			}
   
   			for (Path root : fileSystem.getRootDirectories()) {
   				try (Stream<Path> files = Files.walk(root)) {
   					/*
   					 * LICENSE or NOTICE files found outside of the META-INF directory are most likely shading mistakes (we are including the files from other dependencies, thus providing an invalid LICENSE file)
   					 *
   					 * <p>In such a case, we recommend updating the shading exclusions, and adding the license file to META-INF/licenses.
   					 */
   					final List<String> filesWithIssues = files
   							.filter(path -> !path.equals(root))
   							.filter(path -> path.getFileName().toString().toLowerCase().contains("license"))
   							.filter(path -> !Files.isDirectory(path)) // ignore directories, e.g. "license/"
   							.filter(path -> !path.getFileName().toString().endsWith(".class")) // some class files contain LICENSE in their name
   							.filter(path -> !path.getFileName().toString().endsWith(".ftl")) // a false positive in flink-python
   							.map(Path::toString)
   							.filter(path -> !path.contains("META-INF")) // license files in META-INF are expected
   							.filter(path -> !path.endsWith("web/3rdpartylicenses.txt")) // a false positive in flink-runtime-web
   							.collect(Collectors.toList());
   
   					numSevereIssues += filesWithIssues.size();
   					for (String fileWithIssue : filesWithIssues) {
   						LOG.error("Jar file {} contains a LICENSE file in an unexpected location: {}", file, fileWithIssue);
   					}
   				}
   			}
   		}
   
   		return numSevereIssues;
   ```




----------------------------------------------------------------
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