提交 39ef9593 编写于 作者: C Chesnay Schepler

[FLINK-16163][build] Migrate to flink-shaded-zookeeper

上级 ee7fb5e4
......@@ -92,6 +92,11 @@ under the License.
<artifactId>flink-shaded-jackson</artifactId>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-shaded-zookeeper-3</artifactId>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
......@@ -234,20 +239,6 @@ under the License.
</exclusions>
</dependency>
<!-- Curator and ZooKeeper - we explicitly add ZooKeeper here as
well to make sure our managed version is used -->
<dependency>
<groupId>org.apache.zookeeper</groupId>
<artifactId>zookeeper</artifactId>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-shaded-curator</artifactId>
<version>${project.version}</version>
</dependency>
<!-- Lz4 compression library -->
<dependency>
<groupId>org.lz4</groupId>
......@@ -522,14 +513,12 @@ under the License.
<configuration>
<artifactSet>
<includes combine.children="append">
<include>org.apache.flink:flink-shaded-curator</include>
<!-- add akka, akka's netty, akka uncommon math -->
<!-- we can do this only because our own netty dependency is
already externally shaded (flink-shaded-netty) -->
<include>com.typesafe.akka:akka-remote_*</include>
<include>io.netty:netty</include>
<include>org.uncommons.maths:uncommons-maths</include>
<include>org.apache.zookeeper:*</include>
</includes>
</artifactSet>
<relocations combine.children="append">
......@@ -541,25 +530,6 @@ under the License.
<pattern>org.uncommons.math</pattern>
<shadedPattern>org.apache.flink.shaded.akka.org.uncommons.math</shadedPattern>
</relocation>
<relocation>
<pattern>org.apache.curator</pattern>
<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.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.zookeeper.org.apache.zookeeper.jute</shadedPattern>
</relocation>
</relocations>
<filters>
<filter>
......
......@@ -18,8 +18,8 @@
package org.apache.flink.runtime.checkpoint;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.state.ConnectionState;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
import javax.annotation.Nullable;
......
......@@ -18,8 +18,8 @@
package org.apache.flink.runtime.checkpoint;
import org.apache.curator.framework.state.ConnectionState;
import org.apache.curator.framework.state.ConnectionStateListener;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
import java.util.Optional;
......
......@@ -21,10 +21,11 @@ package org.apache.flink.runtime.checkpoint;
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.shared.SharedCount;
import org.apache.curator.framework.recipes.shared.VersionedValue;
import org.apache.curator.framework.state.ConnectionState;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.shared.SharedCount;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.shared.VersionedValue;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......
......@@ -23,7 +23,7 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
import org.apache.flink.runtime.util.ZooKeeperUtils;
import org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import java.util.concurrent.Executor;
......
......@@ -43,7 +43,7 @@ import org.apache.flink.util.ConfigurationException;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.InstantiationUtil;
import org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import java.net.InetAddress;
import java.net.UnknownHostException;
......
......@@ -23,7 +23,7 @@ import org.apache.flink.runtime.highavailability.ClientHighAvailabilityServices;
import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
import org.apache.flink.runtime.util.ZooKeeperUtils;
import org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import javax.annotation.Nonnull;
......
......@@ -33,9 +33,10 @@ import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
import org.apache.flink.runtime.util.ZooKeeperUtils;
import org.apache.flink.util.ExceptionUtils;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.utils.ZKPaths;
import org.apache.zookeeper.KeeperException;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.utils.ZKPaths;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......
......@@ -23,8 +23,9 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.runtime.highavailability.RunningJobsRegistry;
import org.apache.curator.framework.CuratorFramework;
import org.apache.zookeeper.data.Stat;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......
......@@ -25,12 +25,13 @@ import org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.FlinkException;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
import org.apache.curator.utils.ZKPaths;
import org.apache.zookeeper.KeeperException;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.PathChildrenCache;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
import org.apache.flink.shaded.curator4.org.apache.curator.utils.ZKPaths;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......
......@@ -22,18 +22,19 @@ import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.Preconditions;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.api.UnhandledErrorListener;
import org.apache.curator.framework.recipes.cache.ChildData;
import org.apache.curator.framework.recipes.cache.NodeCache;
import org.apache.curator.framework.recipes.cache.NodeCacheListener;
import org.apache.curator.framework.recipes.leader.LeaderLatch;
import org.apache.curator.framework.recipes.leader.LeaderLatchListener;
import org.apache.curator.framework.state.ConnectionState;
import org.apache.curator.framework.state.ConnectionStateListener;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatch;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.CreateMode;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......
......@@ -21,13 +21,14 @@ package org.apache.flink.runtime.leaderretrieval;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.Preconditions;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.api.UnhandledErrorListener;
import org.apache.curator.framework.recipes.cache.ChildData;
import org.apache.curator.framework.recipes.cache.NodeCache;
import org.apache.curator.framework.recipes.cache.NodeCacheListener;
import org.apache.curator.framework.state.ConnectionState;
import org.apache.curator.framework.state.ConnectionStateListener;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.UnhandledErrorListener;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionStateListener;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......
......@@ -39,15 +39,16 @@ import org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore;
import org.apache.flink.runtime.zookeeper.filesystem.FileSystemStateStorageHelper;
import org.apache.flink.util.Preconditions;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFrameworkFactory;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.ACLProvider;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.imps.DefaultACLProvider;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.PathChildrenCache;
import org.apache.flink.shaded.curator4.org.apache.curator.retry.ExponentialBackoffRetry;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.ZooDefs;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.ACL;
import org.apache.commons.lang3.StringUtils;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.CuratorFrameworkFactory;
import org.apache.curator.framework.api.ACLProvider;
import org.apache.curator.framework.imps.DefaultACLProvider;
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
import org.apache.curator.retry.ExponentialBackoffRetry;
import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.data.ACL;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......
......@@ -22,12 +22,12 @@ import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.configuration.IllegalConfigurationException;
import org.apache.flink.runtime.util.EnvironmentInformation;
import org.apache.zookeeper.server.ServerConfig;
import org.apache.zookeeper.server.ZooKeeperServer;
import org.apache.zookeeper.server.ZooKeeperServerMain;
import org.apache.zookeeper.server.quorum.QuorumPeer;
import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
import org.apache.zookeeper.server.quorum.QuorumPeerMain;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.server.ServerConfig;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.server.ZooKeeperServer;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.server.ZooKeeperServerMain;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.server.quorum.QuorumPeer;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.server.quorum.QuorumPeerConfig;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.server.quorum.QuorumPeerMain;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......
......@@ -18,7 +18,8 @@
package org.apache.flink.runtime.zookeeper;
import org.apache.curator.framework.recipes.shared.SharedCount;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.shared.SharedCount;
import org.apache.flink.util.Preconditions;
import java.io.IOException;
......
......@@ -18,7 +18,8 @@
package org.apache.flink.runtime.zookeeper;
import org.apache.curator.framework.recipes.shared.SharedValue;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.shared.SharedValue;
import org.apache.flink.util.Preconditions;
import java.io.IOException;
......
......@@ -23,11 +23,12 @@ import org.apache.flink.runtime.state.RetrievableStateHandle;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.InstantiationUtil;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.utils.ZKPaths;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.data.Stat;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.utils.ZKPaths;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.CreateMode;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
......
......@@ -22,9 +22,9 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.util.ZooKeeperUtils;
import org.apache.flink.util.Preconditions;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.shared.SharedCount;
import org.apache.curator.framework.recipes.shared.SharedValue;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.shared.SharedCount;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.shared.SharedValue;
import java.io.Serializable;
......
......@@ -18,7 +18,8 @@
package org.apache.flink.runtime.zookeeper;
import org.apache.curator.framework.recipes.shared.VersionedValue;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.shared.VersionedValue;
import org.apache.flink.util.Preconditions;
/**
......
......@@ -8,7 +8,6 @@ This project bundles the following dependencies under the Apache Software Licens
- com.typesafe.akka:akka-remote_2.11:2.5.21
- io.netty:netty:3.10.6.Final
- org.apache.zookeeper:zookeeper:3.4.10
- org.uncommons.maths:uncommons-maths:1.2.2a
This project bundles io.netty:netty:3.10.6.Final from which it inherits the following notices:
......
......@@ -22,7 +22,8 @@ import org.apache.flink.api.common.JobStatus;
import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment;
import org.apache.flink.util.TestLogger;
import org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.Test;
......
......@@ -25,8 +25,9 @@ import org.apache.flink.runtime.util.ZooKeeperUtils;
import org.apache.flink.runtime.zookeeper.ZooKeeperResource;
import org.apache.flink.util.TestLogger;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.state.ConnectionState;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.state.ConnectionState;
import org.junit.Rule;
import org.junit.Test;
......
......@@ -27,8 +27,9 @@ import org.apache.flink.runtime.zookeeper.RetrievableStateStorageHelper;
import org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore;
import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment;
import org.apache.curator.framework.CuratorFramework;
import org.apache.zookeeper.data.Stat;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.Test;
......
......@@ -27,12 +27,13 @@ import org.apache.flink.runtime.zookeeper.RetrievableStateStorageHelper;
import org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore;
import org.apache.flink.util.TestLogger;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.api.BackgroundCallback;
import org.apache.curator.framework.api.CuratorEvent;
import org.apache.curator.framework.api.CuratorEventType;
import org.apache.curator.framework.api.ErrorListenerPathable;
import org.apache.curator.utils.EnsurePath;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.BackgroundCallback;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.CuratorEvent;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.CuratorEventType;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.ErrorListenerPathable;
import org.apache.flink.shaded.curator4.org.apache.curator.utils.EnsurePath;
import org.junit.Test;
import org.mockito.Mockito;
import org.mockito.invocation.InvocationOnMock;
......
......@@ -28,7 +28,8 @@ import org.apache.flink.runtime.zookeeper.ZooKeeperResource;
import org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore;
import org.apache.flink.util.TestLogger;
import org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.hamcrest.Matchers;
import org.junit.ClassRule;
import org.junit.Test;
......
......@@ -56,7 +56,8 @@ import org.apache.flink.runtime.zookeeper.ZooKeeperResource;
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.TestLogger;
import org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.junit.After;
import org.junit.Before;
import org.junit.ClassRule;
......
......@@ -34,9 +34,10 @@ import org.apache.flink.runtime.zookeeper.ZooKeeperResource;
import org.apache.flink.util.TestLogger;
import org.apache.flink.util.function.ThrowingConsumer;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.CuratorFrameworkFactory;
import org.apache.curator.retry.RetryNTimes;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFrameworkFactory;
import org.apache.flink.shaded.curator4.org.apache.curator.retry.RetryNTimes;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
......
......@@ -27,8 +27,9 @@ import org.apache.flink.runtime.zookeeper.ZooKeeperResource;
import org.apache.flink.runtime.zookeeper.ZooKeeperStateHandleStore;
import org.apache.flink.util.TestLogger;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.PathChildrenCache;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
......
......@@ -30,8 +30,9 @@ import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment;
import org.apache.flink.util.InstantiationUtil;
import org.apache.flink.util.TestLogger;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.cache.PathChildrenCache;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.PathChildrenCache;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.Test;
......
......@@ -25,7 +25,8 @@ import org.apache.flink.runtime.testingUtils.TestingUtils;
import org.apache.flink.runtime.util.ZooKeeperUtils;
import org.apache.flink.util.TestLogger;
import org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.curator.test.TestingServer;
import org.junit.After;
import org.junit.Before;
......
......@@ -25,14 +25,15 @@ import org.apache.flink.runtime.leaderretrieval.ZooKeeperLeaderRetrievalService;
import org.apache.flink.runtime.util.ZooKeeperUtils;
import org.apache.flink.util.TestLogger;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.api.CreateBuilder;
import org.apache.curator.framework.recipes.cache.ChildData;
import org.apache.curator.framework.recipes.cache.NodeCache;
import org.apache.curator.framework.recipes.cache.NodeCacheListener;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.api.CreateBuilder;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.ChildData;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCache;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.NodeCacheListener;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.CreateMode;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
import org.apache.curator.test.TestingServer;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
......@@ -409,23 +410,7 @@ public class ZooKeeperLeaderElectionTest extends TestLogger {
try {
client = spy(ZooKeeperUtils.startCuratorFramework(configuration));
Answer<CreateBuilder> answer = new Answer<CreateBuilder>() {
private int counter = 0;
@Override
public CreateBuilder answer(InvocationOnMock invocation) throws Throwable {
counter++;
// at first we have to create the leader latch, there it mustn't fail yet
if (counter < 2) {
return (CreateBuilder) invocation.callRealMethod();
} else {
return mockCreateBuilder;
}
}
};
doAnswer(answer).when(client).create();
doAnswer(invocation -> mockCreateBuilder).when(client).create();
when(
mockCreateBuilder
......
......@@ -32,7 +32,8 @@ import org.apache.flink.runtime.util.LeaderRetrievalUtils;
import org.apache.flink.runtime.util.ZooKeeperUtils;
import org.apache.flink.util.TestLogger;
import org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.curator.test.TestingServer;
import org.junit.After;
import org.junit.Before;
......
......@@ -26,8 +26,9 @@ import org.apache.flink.runtime.util.ZooKeeperUtils;
import org.apache.flink.util.InstantiationUtil;
import org.apache.flink.util.TestLogger;
import org.apache.curator.framework.CuratorFramework;
import org.apache.zookeeper.data.Stat;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.data.Stat;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.Test;
......
......@@ -22,11 +22,12 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.HighAvailabilityOptions;
import org.apache.flink.runtime.util.ZooKeeperUtils;
import org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator4.org.apache.curator.utils.ZKPaths;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;
import org.apache.curator.test.TestingCluster;
import org.apache.curator.test.TestingServer;
import org.apache.curator.utils.ZKPaths;
import org.apache.zookeeper.KeeperException;
import javax.annotation.Nullable;
......
<?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.11-SNAPSHOT</version>
<relativePath>..</relativePath>
</parent>
<artifactId>flink-shaded-curator</artifactId>
<name>flink-shaded-curator</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>
flink-shaded-curator
Copyright 2014-2019 The Apache Software Foundation
This product includes software developed at
The Apache Software Foundation (http://www.apache.org/).
This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt)
- com.google.guava:guava:16.0.1
- org.apache.curator:curator-client:2.12.0
- org.apache.curator:curator-framework:2.12.0
- org.apache.curator:curator-recipes:2.12.0
......@@ -53,7 +53,6 @@ under the License.
<!-- Dummy module to force execution of the Maven Shade plugin (see Shade plugin below) -->
<module>tools/force-shading</module>
<module>flink-annotations</module>
<module>flink-shaded-curator</module>
<module>flink-core</module>
<module>flink-java</module>
<module>flink-scala</module>
......@@ -338,6 +337,12 @@ under the License.
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-shaded-zookeeper-3</artifactId>
<version>${zookeeper.version}-${flink.shaded.version}</version>
</dependency>
<!-- This manages the 'javax.annotation' annotations (JSR305) -->
<dependency>
<groupId>com.google.code.findbugs</groupId>
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册