[FLINK-7778] [build] Shade ZooKeeper dependency (part 2) This closes #4927
Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d368a07a Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d368a07a Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d368a07a Branch: refs/heads/master Commit: d368a07a2146c45a7f994cab0bb55524432a843d Parents: 4d02823 Author: zentol <[email protected]> Authored: Tue Oct 31 11:26:48 2017 +0100 Committer: Stephan Ewen <[email protected]> Committed: Thu Nov 2 16:51:25 2017 +0100 ---------------------------------------------------------------------- .../store/ZooKeeperMesosWorkerStore.java | 11 ++- flink-runtime/pom.xml | 27 +++--- .../runtime/zookeeper/ZookeeperAccess.java | 66 +++++++++++++++ flink-shaded-curator-recipes/pom.xml | 88 ++++++++++++++++++++ flink-shaded-hadoop/pom.xml | 5 -- pom.xml | 1 + tools/travis_mvn_watchdog.sh | 16 ++++ 7 files changed, 188 insertions(+), 26 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/flink/blob/d368a07a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java ---------------------------------------------------------------------- diff --git a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java index 92e4416..738f99e 100644 --- a/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java +++ b/flink-mesos/src/main/java/org/apache/flink/mesos/runtime/clusterframework/store/ZooKeeperMesosWorkerStore.java @@ -25,17 +25,16 @@ import org.apache.flink.runtime.zookeeper.ZooKeeperSharedCount; import org.apache.flink.runtime.zookeeper.ZooKeeperSharedValue; import org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore; import org.apache.flink.runtime.zookeeper.ZooKeeperVersionedValue; +import org.apache.flink.runtime.zookeeper.ZookeeperAccess; import org.apache.flink.util.FlinkException; import org.apache.mesos.Protos; -import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.ArrayList; import java.util.Collections; -import java.util.ConcurrentModificationException; import java.util.List; import scala.Option; @@ -218,15 +217,15 @@ public class ZooKeeperMesosWorkerStore implements MesosWorkerStore { try { workersInZooKeeper.addAndLock(path, worker); LOG.debug("Added {} in ZooKeeper.", worker); - } catch (KeeperException.NodeExistsException ex) { - throw new ConcurrentModificationException("ZooKeeper unexpectedly modified", ex); + } catch (Exception ex) { + throw ZookeeperAccess.wrapIfZooKeeperNodeExistsException(ex); } } else { try { workersInZooKeeper.replace(path, currentVersion, worker); LOG.debug("Updated {} in ZooKeeper.", worker); - } catch (KeeperException.NoNodeException ex) { - throw new ConcurrentModificationException("ZooKeeper unexpectedly modified", ex); + } catch (Exception ex) { + throw ZookeeperAccess.wrapIfZooKeeperNoNodeException(ex); } } } http://git-wip-us.apache.org/repos/asf/flink/blob/d368a07a/flink-runtime/pom.xml ---------------------------------------------------------------------- diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml index 46990a9..481de6d 100644 --- a/flink-runtime/pom.xml +++ b/flink-runtime/pom.xml @@ -178,17 +178,9 @@ under the License. </dependency> <dependency> - <groupId>org.apache.curator</groupId> - <artifactId>curator-recipes</artifactId> - <version>${curator.version}</version> - <exclusions> - <!-- curator shades guava, but still has a dependency on it. --> - <!-- We can safely exclude Guava here --> - <exclusion> - <groupId>com.google.guava</groupId> - <artifactId>guava</artifactId> - </exclusion> - </exclusions> + <groupId>org.apache.flink</groupId> + <artifactId>flink-shaded-curator-recipes</artifactId> + <version>${project.version}</version> </dependency> <!-- test dependencies --> @@ -448,7 +440,6 @@ under the License. <include>com.typesafe.akka:akka-remote_*</include> <include>io.netty:netty</include> <include>org.uncommons.maths:uncommons-maths</include> - <include>org.apache.curator:*</include> <include>org.apache.zookeeper:*</include> </includes> </artifactSet> @@ -463,16 +454,22 @@ under the License. </relocation> <relocation> <pattern>org.apache.curator</pattern> - <shadedPattern>org.apache.flink.shaded.org.apache.curator</shadedPattern> + <shadedPattern>org.apache.flink.shaded.curator.org.apache.curator</shadedPattern> + <excludes> + <!-- Do not relocate curator-test. This leads to problems for downstream + users of runtime test classes that make use of it as the relocated + dependency is not included in the test-jar.--> + <exclude>org.apache.curator.test.*</exclude> + </excludes> </relocation> <relocation> <pattern>org.apache.zookeeper</pattern> - <shadedPattern>org.apache.flink.shaded.org.apache.zookeeper</shadedPattern> + <shadedPattern>org.apache.flink.shaded.zookeeper.org.apache.zookeeper</shadedPattern> </relocation> <!-- jute is already shaded into the ZooKeeper jar --> <relocation> <pattern>org.apache.jute</pattern> - <shadedPattern>org.apache.flink.shaded.org.apache.zookeeper.jute</shadedPattern> + <shadedPattern>org.apache.flink.shaded.zookeeper.org.apache.zookeeper.jute</shadedPattern> </relocation> </relocations> <filters> http://git-wip-us.apache.org/repos/asf/flink/blob/d368a07a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZookeeperAccess.java ---------------------------------------------------------------------- diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZookeeperAccess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZookeeperAccess.java new file mode 100644 index 0000000..2c6160a --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZookeeperAccess.java @@ -0,0 +1,66 @@ +/* + * 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.runtime.zookeeper; + +import org.apache.zookeeper.KeeperException; + +import java.util.ConcurrentModificationException; + +/** + * Utility class providing access to relocated zookeeper classes. + * + * <p>This class is necessary as flink-runtime relocates its ZooKeeper dependency. + * Other modules may still depend on this dependency but will encounter a ClassNotFoundException + * on access as they don't apply the relocation pattern of flink-runtime. + */ +public final class ZookeeperAccess { + + private ZookeeperAccess(){ + } + + /** + * Wraps and returns the given exception in a {@link ConcurrentModificationException} if it is a + * {@link org.apache.zookeeper.KeeperException.NodeExistsException}. Otherwise the + * given exception is returned. + * + * @param ex exception to wrap + * @return wrapping ConcurrentModificationException if it is a NodeExistsException, otherwise the given exception + */ + public static Exception wrapIfZooKeeperNodeExistsException(Exception ex) { + if (ex instanceof KeeperException.NodeExistsException) { + return new ConcurrentModificationException("ZooKeeper unexpectedly modified", ex); + } + return ex; + } + + /** + * Wraps and returns the given exception in a {@link ConcurrentModificationException} if it is a + * {@link org.apache.zookeeper.KeeperException.NoNodeException}. Otherwise the + * given exception is returned. + * + * @param ex exception to wrap + * @return wrapping ConcurrentModificationException if it is a NoNodeException, otherwise the given exception + */ + public static Exception wrapIfZooKeeperNoNodeException(Exception ex) { + if (ex instanceof KeeperException.NoNodeException) { + return new ConcurrentModificationException("ZooKeeper unexpectedly modified", ex); + } + return ex; + } +} http://git-wip-us.apache.org/repos/asf/flink/blob/d368a07a/flink-shaded-curator-recipes/pom.xml ---------------------------------------------------------------------- diff --git a/flink-shaded-curator-recipes/pom.xml b/flink-shaded-curator-recipes/pom.xml new file mode 100644 index 0000000..06d3821 --- /dev/null +++ b/flink-shaded-curator-recipes/pom.xml @@ -0,0 +1,88 @@ +<?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.flink</groupId> + <artifactId>flink-parent</artifactId> + <version>1.4-SNAPSHOT</version> + <relativePath>..</relativePath> + </parent> + + <artifactId>flink-shaded-curator-recipes</artifactId> + <name>flink-shaded-curator-recipes</name> + + <packaging>jar</packaging> + + + <dependencies> + <dependency> + <groupId>org.apache.curator</groupId> + <artifactId>curator-recipes</artifactId> + <version>${curator.version}</version> + </dependency> + </dependencies> + + <build> + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-shade-plugin</artifactId> + <executions> + <execution> + <id>shade-flink</id> + <phase>package</phase> + <goals> + <goal>shade</goal> + </goals> + <configuration> + <artifactSet> + <includes> + <include>com.google.guava:guava</include> + <include>org.apache.curator:*</include> + </includes> + </artifactSet> + <filters> + <filter> + <artifact>com.google.guava:guava</artifact> + <!-- Shade guava classes that are not included by curator --> + <includes> + <include>com/google/common/base/Function.class</include> + <include>com/google/common/base/Predicate.class</include> + <include>com/google/common/reflect/TypeToken.class</include> + </includes> + </filter> + </filters> + <relocations> + <relocation> + <pattern>com.google.common</pattern> + <shadedPattern>org.apache.flink.curator.shaded.com.google.common</shadedPattern> + </relocation> + </relocations> + </configuration> + </execution> + </executions> + </plugin> + </plugins> + </build> +</project> http://git-wip-us.apache.org/repos/asf/flink/blob/d368a07a/flink-shaded-hadoop/pom.xml ---------------------------------------------------------------------- diff --git a/flink-shaded-hadoop/pom.xml b/flink-shaded-hadoop/pom.xml index f4b0816..ba90fc9 100644 --- a/flink-shaded-hadoop/pom.xml +++ b/flink-shaded-hadoop/pom.xml @@ -53,11 +53,6 @@ under the License. <scope>provided</scope> </dependency> <dependency> - <groupId>org.apache.zookeeper</groupId> - <artifactId>zookeeper</artifactId> - <scope>provided</scope> - </dependency> - <dependency> <groupId>org.apache.avro</groupId> <artifactId>avro</artifactId> <scope>provided</scope> http://git-wip-us.apache.org/repos/asf/flink/blob/d368a07a/pom.xml ---------------------------------------------------------------------- diff --git a/pom.xml b/pom.xml index a5f99f7..faddab4 100644 --- a/pom.xml +++ b/pom.xml @@ -54,6 +54,7 @@ under the License. <module>tools/force-shading</module> <module>flink-annotations</module> <module>flink-shaded-hadoop</module> + <module>flink-shaded-curator-recipes</module> <module>flink-core</module> <module>flink-java</module> <module>flink-java8</module> http://git-wip-us.apache.org/repos/asf/flink/blob/d368a07a/tools/travis_mvn_watchdog.sh ---------------------------------------------------------------------- diff --git a/tools/travis_mvn_watchdog.sh b/tools/travis_mvn_watchdog.sh index b7e3709..978bc9f 100755 --- a/tools/travis_mvn_watchdog.sh +++ b/tools/travis_mvn_watchdog.sh @@ -325,6 +325,22 @@ check_shaded_artifacts() { return 1 fi + ZOOKEEPER=`cat allClasses | grep '^org/apache/zookeeper' | wc -l` + if [ "$ZOOKEEPER" != "0" ]; then + echo "==============================================================================" + echo "Detected '$ZOOKEEPER' unshaded org.apache.zookeeper classes in fat jar" + echo "==============================================================================" + return 1 + fi + + CURATOR=`cat allClasses | grep '^org/apache/curator' | wc -l` + if [ "$CURATOR" != "0" ]; then + echo "==============================================================================" + echo "Detected '$CURATOR' unshaded org.apache.curator classes in fat jar" + echo "==============================================================================" + return 1 + fi + FLINK_PYTHON=`cat allClasses | grep '^org/apache/flink/python' | wc -l` if [ "$FLINK_PYTHON" != "0" ]; then echo "=============================================================================="
