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

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

commit 4fa80bf27b5501e87c9abcb2004fac8083c6d660
Author: Chesnay Schepler <ches...@apache.org>
AuthorDate: Thu Jun 23 12:19:49 2022 +0200

    [FLINK-28203][build] Add ShadeOptionalChecker
---
 tools/ci/compile.sh                                |   4 +
 .../tools/ci/optional/ShadeOptionalChecker.java    | 266 +++++++++++++++++++++
 .../tools/ci/utils/shared/DependencyTree.java      |  20 +-
 .../ci/optional/ShadeOptionalCheckerTest.java      | 181 ++++++++++++++
 tools/ci/verify_bundled_optional.sh                |  48 ++++
 5 files changed, 515 insertions(+), 4 deletions(-)

diff --git a/tools/ci/compile.sh b/tools/ci/compile.sh
index 2fb52f394de..c2c68be7a47 100755
--- a/tools/ci/compile.sh
+++ b/tools/ci/compile.sh
@@ -90,6 +90,10 @@ if [ $EXIT_CODE != 0 ] ; then
   exit $EXIT_CODE
 fi
 
+echo "============ Checking bundled dependencies marked as optional 
============"
+
+${CI_DIR}/verify_bundled_optional.sh $MVN_CLEAN_COMPILE_OUT "$CI_DIR" "$(pwd)" 
|| exit $?
+
 echo "============ Checking scala suffixes ============"
 
 ${CI_DIR}/verify_scala_suffixes.sh "$CI_DIR" "$(pwd)" || exit $?
diff --git 
a/tools/ci/flink-ci-tools/src/main/java/org/apache/flink/tools/ci/optional/ShadeOptionalChecker.java
 
b/tools/ci/flink-ci-tools/src/main/java/org/apache/flink/tools/ci/optional/ShadeOptionalChecker.java
new file mode 100644
index 00000000000..a15cdd73afb
--- /dev/null
+++ 
b/tools/ci/flink-ci-tools/src/main/java/org/apache/flink/tools/ci/optional/ShadeOptionalChecker.java
@@ -0,0 +1,266 @@
+/*
+ * 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.optional;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.tools.ci.utils.dependency.DependencyParser;
+import org.apache.flink.tools.ci.utils.shade.ShadeParser;
+import org.apache.flink.tools.ci.utils.shared.Dependency;
+import org.apache.flink.tools.ci.utils.shared.DependencyTree;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Verifies that all dependencies bundled with the shade-plugin are marked as 
optional in the pom.
+ * This ensures compatibility with later maven versions and in general 
simplifies dependency
+ * management as transitivity is no longer dependent on the shade-plugin.
+ *
+ * <p>In Maven 3.3 the dependency tree was made immutable at runtime, and thus 
can no longer be
+ * changed by the shade plugin. The plugin would usually remove a dependency 
from the tree when it
+ * is being bundled (known as dependency reduction). While dependency 
reduction still works for the
+ * published poms (== what users consume) since it can still change the 
content of the final pom,
+ * while developing Flink it no longer works. This breaks plenty of things, 
since suddenly a bunch
+ * of dependencies are still visible to downstream modules that weren't before.
+ *
+ * <p>To workaround this we mark all dependencies that we bundle as optional; 
this makes them
+ * non-transitive. To a downstream module, behavior-wise a non-transitive 
dependency is identical to
+ * a removed dependency.
+ *
+ * <p>This checker analyzes the bundled dependencies (based on the 
shade-plugin output) and the set
+ * of dependencies (based on the dependency plugin) to detect cases where a 
dependency is not marked
+ * as optional as it should.
+ *
+ * <p>The enforced rule is rather simple: Any dependency that is bundled, or 
any of its parents,
+ * must show up as optional in the dependency tree. The parent clause is 
required to cover cases
+ * where a module has 2 paths to a bundled dependency. If a module depends on 
A1/A2, each depending
+ * on B, with A1 and B being bundled, then even if A1 is marked as optional B 
is still shown as a
+ * non-optional dependency (because the non-optional A2 still needs it!).
+ */
+public class ShadeOptionalChecker {
+    private static final Logger LOG = 
LoggerFactory.getLogger(ShadeOptionalChecker.class);
+
+    public static void main(String[] args) throws IOException {
+        if (args.length < 2) {
+            System.out.println(
+                    "Usage: ShadeOptionalChecker <pathShadeBuildOutput> 
<pathMavenDependencyOutput>");
+            System.exit(1);
+        }
+
+        final Path shadeOutputPath = Paths.get(args[0]);
+        final Path dependencyOutputPath = Paths.get(args[1]);
+
+        final Map<String, Set<Dependency>> bundledDependenciesByModule =
+                ShadeParser.parseShadeOutput(shadeOutputPath);
+        final Map<String, DependencyTree> dependenciesByModule =
+                
DependencyParser.parseDependencyTreeOutput(dependencyOutputPath);
+
+        final Map<String, Set<Dependency>> violations =
+                checkOptionalFlags(bundledDependenciesByModule, 
dependenciesByModule);
+
+        if (!violations.isEmpty()) {
+            LOG.error(
+                    "{} modules bundle in total {} dependencies without them 
being marked as optional in the pom.",
+                    violations.keySet().size(),
+                    
violations.values().stream().mapToInt(Collection::size).sum());
+            LOG.error(
+                    "\tIn order for shading to properly work within Flink we 
require all bundled dependencies to be marked as optional in the pom.");
+            LOG.error(
+                    "\tFor verification purposes we require the dependency 
tree from the dependency-plugin to show the dependency as either:");
+            LOG.error("\t\ta) an optional dependency,");
+            LOG.error("\t\tb) a transitive dependency of another optional 
dependency.");
+            LOG.error(
+                    "\tIn most cases adding 
'<optional>${flink.markBundledAsOptional}</optional>' to the bundled dependency 
is sufficient.");
+            LOG.error(
+                    "\tThere are some edge cases where a transitive dependency 
might be associated with the \"wrong\" dependency in the tree, for example if a 
test dependency also requires it.");
+            LOG.error(
+                    "\tIn such cases you need to adjust the poms so that the 
dependency shows up in the right spot. This may require adding an explicit 
dependency (Management) entry, excluding dependencies, or at times even 
reordering dependencies in the pom.");
+            LOG.error(
+                    "\tSee the Dependencies page in the wiki for details: 
https://cwiki.apache.org/confluence/display/FLINK/Dependencies";);
+
+            for (String moduleWithViolations : violations.keySet()) {
+                final Collection<Dependency> dependencyViolations =
+                        violations.get(moduleWithViolations);
+                LOG.error(
+                        "\tModule {} ({} violation{}):",
+                        moduleWithViolations,
+                        dependencyViolations.size(),
+                        dependencyViolations.size() == 1 ? "" : "s");
+                for (Dependency dependencyViolation : dependencyViolations) {
+                    LOG.error("\t\t{}", dependencyViolation);
+                }
+            }
+
+            System.exit(1);
+        }
+    }
+
+    private static Map<String, Set<Dependency>> checkOptionalFlags(
+            Map<String, Set<Dependency>> bundledDependenciesByModule,
+            Map<String, DependencyTree> dependenciesByModule) {
+
+        final Map<String, Set<Dependency>> allViolations = new HashMap<>();
+
+        for (String module : bundledDependenciesByModule.keySet()) {
+            LOG.debug("Checking module '{}'.", module);
+            if (!dependenciesByModule.containsKey(module)) {
+                throw new IllegalStateException(
+                        String.format(
+                                "Module %s listed by shade-plugin, but not 
dependency-plugin.",
+                                module));
+            }
+
+            final Collection<Dependency> bundledDependencies =
+                    bundledDependenciesByModule.get(module);
+            final DependencyTree dependencyTree = 
dependenciesByModule.get(module);
+
+            final Set<Dependency> violations =
+                    checkOptionalFlags(module, bundledDependencies, 
dependencyTree);
+
+            if (violations.isEmpty()) {
+                LOG.info("OK: {}", module);
+            } else {
+                allViolations.put(module, violations);
+            }
+        }
+
+        return allViolations;
+    }
+
+    @VisibleForTesting
+    static Set<Dependency> checkOptionalFlags(
+            String module,
+            Collection<Dependency> bundledDependencies,
+            DependencyTree dependencyTree) {
+
+        bundledDependencies =
+                bundledDependencies.stream()
+                        // force-shading isn't relevant for this check but 
breaks some shortcuts
+                        .filter(
+                                dependency ->
+                                        !dependency
+                                                .getArtifactId()
+                                                
.equals("flink-shaded-force-shading"))
+                        .collect(Collectors.toSet());
+
+        final Set<Dependency> violations = new HashSet<>();
+
+        if (bundledDependencies.isEmpty()) {
+            LOG.debug("\tModule is not bundling any dependencies.");
+            return violations;
+        }
+
+        // The set of dependencies that the module directly depends on and 
which downstream modules
+        // would pull in transitively.
+        //
+        // If this set is empty we do not need to check anything.
+        // This allows us to avoid some edge-cases:
+        //
+        // Assume module M has the following (full) dependency tree, bundling 
dependency 1 and 2:
+        //
+        // +- dependency1 (compile/optional)",
+        // |  \- dependency2 (compile) (implicitly optional because 
dependency1 is optional)
+        // \- dependency3 (test)
+        //    \- dependency2 (compile)
+        //
+        // However, in the dependency plugin output a dependency can only show 
up once, so Maven may
+        // return this:
+        //
+        // +- dependency1 (compile/optional)",
+        // \- dependency3 (test)
+        //    \- dependency2 (compile)
+        //
+        // Given this tree, and knowing that dependency2 is bundled, we would 
draw the conclusion
+        // that dependency2 is missing the optional flag.
+        //
+        // However, because dependency 1/3 are optional/test dependencies they 
are not transitive.
+        // Without any direct transitive dependency nothing can leak through 
to downstream modules,
+        // removing the need to check dependency 2 at all (and in turn, saving 
us from having to
+        // resolve this problem).
+        final List<Dependency> directTransitiveDependencies =
+                dependencyTree.getDirectDependencies().stream()
+                        .filter(
+                                dependency ->
+                                        !(isOptional(dependency)
+                                                || hasProvidedScope(dependency)
+                                                || hasTestScope(dependency)
+                                                || 
isCommonCompileDependency(dependency)))
+                        .collect(Collectors.toList());
+
+        // if nothing would be exposed to downstream modules we exit early to 
reduce noise on CI
+        if (directTransitiveDependencies.isEmpty()) {
+            LOG.debug(
+                    "Skipping deep-check of module {} because all direct 
dependencies are not transitive.",
+                    module);
+            return violations;
+        }
+        LOG.debug(
+                "Running deep-check of module {} because there are direct 
dependencies that are transitive: {}",
+                module,
+                directTransitiveDependencies);
+
+        for (Dependency bundledDependency : bundledDependencies) {
+            LOG.debug("\tChecking dependency '{}'.", bundledDependency);
+
+            final List<Dependency> dependencyPath = 
dependencyTree.getPathTo(bundledDependency);
+
+            final boolean isOptional =
+                    dependencyPath.stream().anyMatch(parent -> 
parent.isOptional().orElse(false));
+
+            if (!isOptional) {
+                violations.add(bundledDependency);
+            }
+        }
+
+        return violations;
+    }
+
+    private static boolean isOptional(Dependency dependency) {
+        return dependency.isOptional().orElse(false);
+    }
+
+    private static boolean hasProvidedScope(Dependency dependency) {
+        return "provided".equals(dependency.getScope().orElse(null));
+    }
+
+    private static boolean hasTestScope(Dependency dependency) {
+        return "test".equals(dependency.getScope().orElse(null));
+    }
+
+    /**
+     * These are compile dependencies that are set up in the root pom. We do 
not require modules to
+     * mark these as optional because all modules depend on them anyway; 
whether they leak through
+     * or not is therefore irrelevant.
+     */
+    private static boolean isCommonCompileDependency(Dependency dependency) {
+        return "flink-shaded-force-shading".equals(dependency.getArtifactId())
+                || "jsr305".equals(dependency.getArtifactId())
+                || "slf4j-api".equals(dependency.getArtifactId());
+    }
+}
diff --git 
a/tools/ci/flink-ci-tools/src/main/java/org/apache/flink/tools/ci/utils/shared/DependencyTree.java
 
b/tools/ci/flink-ci-tools/src/main/java/org/apache/flink/tools/ci/utils/shared/DependencyTree.java
index 4b26121c14c..3b87610d2b5 100644
--- 
a/tools/ci/flink-ci-tools/src/main/java/org/apache/flink/tools/ci/utils/shared/DependencyTree.java
+++ 
b/tools/ci/flink-ci-tools/src/main/java/org/apache/flink/tools/ci/utils/shared/DependencyTree.java
@@ -28,6 +28,7 @@ import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import java.util.stream.StreamSupport;
 
@@ -41,25 +42,30 @@ public class DependencyTree {
     private final Map<String, Node> lookup = new LinkedHashMap<>();
     private final List<Node> directDependencies = new ArrayList<>();
 
-    public void addDirectDependency(Dependency dependency) {
+    public DependencyTree addDirectDependency(Dependency dependency) {
         final String key = getKey(dependency);
         if (lookup.containsKey(key)) {
-            return;
+            return this;
         }
         final Node node = new Node(dependency, null);
 
         lookup.put(key, node);
         directDependencies.add(node);
+
+        return this;
     }
 
-    public void addTransitiveDependencyTo(Dependency transitiveDependency, 
Dependency parent) {
+    public DependencyTree addTransitiveDependencyTo(
+            Dependency transitiveDependency, Dependency parent) {
         final String key = getKey(transitiveDependency);
         if (lookup.containsKey(key)) {
-            return;
+            return this;
         }
         final Node node = 
lookup.get(getKey(parent)).addTransitiveDependency(transitiveDependency);
 
         lookup.put(key, node);
+
+        return this;
     }
 
     private static final class Node {
@@ -83,6 +89,12 @@ public class DependencyTree {
         }
     }
 
+    public List<Dependency> getDirectDependencies() {
+        return directDependencies.stream()
+                .map(node -> node.dependency)
+                .collect(Collectors.toList());
+    }
+
     public List<Dependency> getPathTo(Dependency dependency) {
         final LinkedList<Dependency> path = new LinkedList<>();
 
diff --git 
a/tools/ci/flink-ci-tools/src/test/java/org/apache/flink/tools/ci/optional/ShadeOptionalCheckerTest.java
 
b/tools/ci/flink-ci-tools/src/test/java/org/apache/flink/tools/ci/optional/ShadeOptionalCheckerTest.java
new file mode 100644
index 00000000000..edfbbfd4871
--- /dev/null
+++ 
b/tools/ci/flink-ci-tools/src/test/java/org/apache/flink/tools/ci/optional/ShadeOptionalCheckerTest.java
@@ -0,0 +1,181 @@
+/*
+ * 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.optional;
+
+import org.apache.flink.tools.ci.utils.shared.Dependency;
+import org.apache.flink.tools.ci.utils.shared.DependencyTree;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.Set;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+class ShadeOptionalCheckerTest {
+    private static final String MODULE = "module";
+
+    @Test
+    void testNonBundledDependencyIsIgnored() {
+        final Dependency dependency = createMandatoryDependency("a");
+        final Set<Dependency> bundled = Collections.emptySet();
+        final DependencyTree dependencyTree = new 
DependencyTree().addDirectDependency(dependency);
+
+        final Set<Dependency> violations =
+                ShadeOptionalChecker.checkOptionalFlags(MODULE, bundled, 
dependencyTree);
+
+        assertThat(violations).isEmpty();
+    }
+
+    @Test
+    void testNonBundledDependencyIsIgnoredEvenIfOthersAreBundled() {
+        final Dependency dependencyA = createMandatoryDependency("a");
+        final Dependency dependencyB = createMandatoryDependency("B");
+        final Set<Dependency> bundled = Collections.singleton(dependencyB);
+        final DependencyTree dependencyTree =
+                new DependencyTree()
+                        .addDirectDependency(dependencyA)
+                        .addDirectDependency(dependencyB);
+
+        final Set<Dependency> violations =
+                ShadeOptionalChecker.checkOptionalFlags(MODULE, bundled, 
dependencyTree);
+
+        assertThat(violations).containsExactly(dependencyB);
+    }
+
+    @Test
+    void testDirectBundledOptionalDependencyIsAccepted() {
+        final Dependency dependency = createOptionalDependency("a");
+        final Set<Dependency> bundled = Collections.singleton(dependency);
+        final DependencyTree dependencyTree = new 
DependencyTree().addDirectDependency(dependency);
+
+        final Set<Dependency> violations =
+                ShadeOptionalChecker.checkOptionalFlags(MODULE, bundled, 
dependencyTree);
+
+        assertThat(violations).isEmpty();
+    }
+
+    @Test
+    void testDirectBundledDependencyMustBeOptional() {
+        final Dependency dependency = createMandatoryDependency("a");
+        final Set<Dependency> bundled = Collections.singleton(dependency);
+        final DependencyTree dependencyTree = new 
DependencyTree().addDirectDependency(dependency);
+
+        final Set<Dependency> violations =
+                ShadeOptionalChecker.checkOptionalFlags(MODULE, bundled, 
dependencyTree);
+
+        assertThat(violations).containsExactly(dependency);
+    }
+
+    @Test
+    void testTransitiveBundledOptionalDependencyIsAccepted() {
+        final Dependency dependencyA = createMandatoryDependency("a");
+        final Dependency dependencyB = createOptionalDependency("b");
+        final Set<Dependency> bundled = Collections.singleton(dependencyB);
+        final DependencyTree dependencyTree =
+                new DependencyTree()
+                        .addDirectDependency(dependencyA)
+                        .addTransitiveDependencyTo(dependencyB, dependencyA);
+
+        final Set<Dependency> violations =
+                ShadeOptionalChecker.checkOptionalFlags(MODULE, bundled, 
dependencyTree);
+
+        assertThat(violations).isEmpty();
+    }
+
+    @Test
+    void testTransitiveBundledDependencyMustBeOptional() {
+        final Dependency dependencyA = createMandatoryDependency("a");
+        final Dependency dependencyB = createMandatoryDependency("b");
+        final Set<Dependency> bundled = Collections.singleton(dependencyB);
+        final DependencyTree dependencyTree =
+                new DependencyTree()
+                        .addDirectDependency(dependencyA)
+                        .addTransitiveDependencyTo(dependencyB, dependencyA);
+
+        final Set<Dependency> violations =
+                ShadeOptionalChecker.checkOptionalFlags(MODULE, bundled, 
dependencyTree);
+
+        assertThat(violations).containsExactly(dependencyB);
+    }
+
+    @Test
+    void testTransitiveBundledDependencyMayNotBeOptionalIfParentIsOptional() {
+        final Dependency dependencyA = createOptionalDependency("a");
+        final Dependency dependencyB = createMandatoryDependency("b");
+        final Set<Dependency> bundled = Collections.singleton(dependencyB);
+        final DependencyTree dependencyTree =
+                new DependencyTree()
+                        .addDirectDependency(dependencyA)
+                        .addTransitiveDependencyTo(dependencyB, dependencyA);
+
+        final Set<Dependency> violations =
+                ShadeOptionalChecker.checkOptionalFlags(MODULE, bundled, 
dependencyTree);
+
+        assertThat(violations).isEmpty();
+    }
+
+    @Test
+    void testTransitiveBundledDependencyMayNotBeOptionalIfParentHasTestScope() 
{
+        final Dependency dependencyA = createTestDependency("a");
+        final Dependency dependencyB = createMandatoryDependency("b");
+        final Set<Dependency> bundled = Collections.singleton(dependencyB);
+        final DependencyTree dependencyTree =
+                new DependencyTree()
+                        .addDirectDependency(dependencyA)
+                        .addTransitiveDependencyTo(dependencyB, dependencyA);
+
+        final Set<Dependency> violations =
+                ShadeOptionalChecker.checkOptionalFlags(MODULE, bundled, 
dependencyTree);
+
+        assertThat(violations).isEmpty();
+    }
+
+    @Test
+    void 
testTransitiveBundledDependencyMayNotBeOptionalIfParentHasProvidedScope() {
+        final Dependency dependencyA = createProvidedDependency("a");
+        final Dependency dependencyB = createMandatoryDependency("b");
+        final Set<Dependency> bundled = Collections.singleton(dependencyB);
+        final DependencyTree dependencyTree =
+                new DependencyTree()
+                        .addDirectDependency(dependencyA)
+                        .addTransitiveDependencyTo(dependencyB, dependencyA);
+
+        final Set<Dependency> violations =
+                ShadeOptionalChecker.checkOptionalFlags(MODULE, bundled, 
dependencyTree);
+
+        assertThat(violations).isEmpty();
+    }
+
+    private static Dependency createMandatoryDependency(String artifactId) {
+        return Dependency.create("groupId", artifactId, "version", null);
+    }
+
+    private static Dependency createOptionalDependency(String artifactId) {
+        return Dependency.create("groupId", artifactId, "version", null, 
"compile", true);
+    }
+
+    private static Dependency createProvidedDependency(String artifactId) {
+        return Dependency.create("groupId", artifactId, "version", null, 
"provided", false);
+    }
+
+    private static Dependency createTestDependency(String artifactId) {
+        return Dependency.create("groupId", artifactId, "version", null, 
"test", false);
+    }
+}
diff --git a/tools/ci/verify_bundled_optional.sh 
b/tools/ci/verify_bundled_optional.sh
new file mode 100755
index 00000000000..db43b320249
--- /dev/null
+++ b/tools/ci/verify_bundled_optional.sh
@@ -0,0 +1,48 @@
+#!/usr/bin/env bash
+
+#
+# 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.
+#
+
+## Checks that all bundled dependencies are marked as optional in the poms
+MVN_CLEAN_COMPILE_OUT=$1
+CI_DIR=$2
+FLINK_ROOT=$3
+
+source "${CI_DIR}/maven-utils.sh"
+
+cd "$FLINK_ROOT" || exit
+
+dependency_plugin_output=${CI_DIR}/optional_dep.txt
+
+run_mvn dependency:tree -B > "${dependency_plugin_output}"
+
+cat "${dependency_plugin_output}"
+
+cd "${CI_DIR}/flink-ci-tools/" || exit
+
+run_mvn exec:java 
-Dexec.mainClass=org.apache.flink.tools.ci.optional.ShadeOptionalChecker 
-Dexec.args=\""${MVN_CLEAN_COMPILE_OUT}" "${dependency_plugin_output}"\"
+EXIT_CODE=$?
+
+if [ $EXIT_CODE != 0 ]; then
+    echo 
"=============================================================================="
+    echo "Optional Check failed. See previous output for details."
+    echo 
"=============================================================================="
+    exit 1
+fi
+
+exit 0
+

Reply via email to