Skip to content

Commit

Permalink
[FLINK-25145][build] Drop ZK 3.4 / Support ZK 3.6
Browse files Browse the repository at this point in the history
  • Loading branch information
zentol committed Feb 1, 2022
1 parent dea2b10 commit 8ddfd59
Show file tree
Hide file tree
Showing 52 changed files with 160 additions and 116 deletions.
11 changes: 11 additions & 0 deletions flink-connectors/flink-connector-hbase-1.4/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -352,6 +352,17 @@ under the License.

</dependencies>

<dependencyManagement>
<dependencies>
<dependency>
<!-- HBase only works with Zookeper 3.4 -->
<groupId>org.apache.zookeeper</groupId>
<artifactId>zookeeper</artifactId>
<version>3.4.14</version>
</dependency>
</dependencies>
</dependencyManagement>

<profiles>
<profile>
<id>java11</id>
Expand Down
11 changes: 11 additions & 0 deletions flink-connectors/flink-connector-hbase-2.2/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -424,6 +424,17 @@ under the License.

</dependencies>

<dependencyManagement>
<dependencies>
<dependency>
<!-- HBase only works with Zookeper 3.4 -->
<groupId>org.apache.zookeeper</groupId>
<artifactId>zookeeper</artifactId>
<version>3.4.14</version>
</dependency>
</dependencies>
</dependencyManagement>

<profiles>
<profile>
<id>java11</id>
Expand Down
11 changes: 11 additions & 0 deletions flink-connectors/flink-sql-connector-hbase-1.4/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,17 @@ under the License.
</dependency>
</dependencies>

<dependencyManagement>
<dependencies>
<dependency>
<!-- HBase only works with Zookeper 3.4 -->
<groupId>org.apache.zookeeper</groupId>
<artifactId>zookeeper</artifactId>
<version>3.4.14</version>
</dependency>
</dependencies>
</dependencyManagement>

<build>
<plugins>
<plugin>
Expand Down
11 changes: 11 additions & 0 deletions flink-connectors/flink-sql-connector-hbase-2.2/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,17 @@ under the License.
</dependency>
</dependencies>

<dependencyManagement>
<dependencies>
<dependency>
<!-- HBase only works with Zookeper 3.4 -->
<groupId>org.apache.zookeeper</groupId>
<artifactId>zookeeper</artifactId>
<version>3.4.14</version>
</dependency>
</dependencies>
</dependencyManagement>

<build>
<plugins>
<plugin>
Expand Down
4 changes: 2 additions & 2 deletions flink-dist/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ under the License.
<packaging>jar</packaging>

<properties>
<zookeeper.optional.version>3.5.9</zookeeper.optional.version>
<zookeeper.optional.version>3.6.3</zookeeper.optional.version>
</properties>

<dependencies>
Expand Down Expand Up @@ -664,7 +664,7 @@ under the License.
<artifactItem>
<groupId>org.apache.flink</groupId>
<artifactId>flink-shaded-zookeeper-3</artifactId>
<version>${zookeeper.optional.version}-14.0</version>
<version>${zookeeper.optional.version}-${flink.shaded.version}</version>
<type>jar</type>
<overWrite>true</overWrite>
</artifactItem>
Expand Down
2 changes: 1 addition & 1 deletion flink-dist/src/main/assemblies/bin.xml
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ under the License.
<outputDirectory>lib</outputDirectory>
<unpack>false</unpack>
<includes>
<include>org.apache.flink:flink-shaded-zookeeper-3:jar:${zookeeper.version}-14.0</include>
<include>org.apache.flink:flink-shaded-zookeeper-3:jar:${zookeeper.version}-${flink.shaded.version}</include>
</includes>
<outputFileNameMapping>flink-shaded-zookeeper-${zookeeper.version}.jar</outputFileNameMapping>
</dependencySet>
Expand Down
2 changes: 1 addition & 1 deletion flink-dist/src/main/assemblies/opt.xml
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,7 @@
</file>

<file>
<source>target/temporary/flink-shaded-zookeeper-3-${zookeeper.optional.version}-14.0.jar</source>
<source>target/temporary/flink-shaded-zookeeper-3-${zookeeper.optional.version}-${flink.shaded.version}.jar</source>
<outputDirectory>opt</outputDirectory>
<destName>flink-shaded-zookeeper-${zookeeper.optional.version}.jar</destName>
<fileMode>0644</fileMode>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -246,7 +246,7 @@ private List<GenericContainer<?>> buildTaskManagerContainers(Path tempDirectory)

private GenericContainer<?> buildZookeeperContainer() {
return configureContainer(
new GenericContainer<>(DockerImageName.parse("zookeeper").withTag("3.4.14")),
new GenericContainer<>(DockerImageName.parse("zookeeper").withTag("3.5.9")),
ZOOKEEPER_HOSTNAME,
"Zookeeper");
}
Expand Down
8 changes: 4 additions & 4 deletions flink-end-to-end-tests/run-nightly-tests.sh
Original file line number Diff line number Diff line change
Expand Up @@ -162,14 +162,14 @@ function run_group_1 {

run_test "Running HA dataset end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_dataset.sh" "skip_check_exceptions"

run_test "Running HA (hashmap, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh hashmap true false 3.4" "skip_check_exceptions"
run_test "Running HA (hashmap, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh hashmap true false 3.6" "skip_check_exceptions"
run_test "Running HA (hashmap, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh hashmap false false 3.5" "skip_check_exceptions"
run_test "Running HA (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh rocks true false 3.4" "skip_check_exceptions"
run_test "Running HA (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh rocks true false 3.6" "skip_check_exceptions"
run_test "Running HA (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh rocks true true 3.5" "skip_check_exceptions"

run_test "Running HA per-job cluster (hashmap, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh hashmap true false 3.4" "skip_check_exceptions"
run_test "Running HA per-job cluster (hashmap, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh hashmap true false 3.6" "skip_check_exceptions"
run_test "Running HA per-job cluster (hashmap, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh hashmap false false 3.5" "skip_check_exceptions"
run_test "Running HA per-job cluster (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh rocks true false 3.4" "skip_check_exceptions"
run_test "Running HA per-job cluster (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh rocks true false 3.6" "skip_check_exceptions"
run_test "Running HA per-job cluster (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_per_job_cluster_datastream.sh rocks true true 3.5" "skip_check_exceptions"
}

Expand Down
2 changes: 1 addition & 1 deletion flink-end-to-end-tests/test-scripts/test_ha_datastream.sh
Original file line number Diff line number Diff line change
Expand Up @@ -98,6 +98,6 @@ function run_ha_test() {
STATE_BACKEND_TYPE=${1:-file}
STATE_BACKEND_FILE_ASYNC=${2:-true}
STATE_BACKEND_ROCKS_INCREMENTAL=${3:-false}
ZOOKEEPER_VERSION=${4:-3.4}
ZOOKEEPER_VERSION=${4:-3.5}

run_test_with_timeout 900 run_ha_test 4 ${STATE_BACKEND_TYPE} ${STATE_BACKEND_FILE_ASYNC} ${STATE_BACKEND_ROCKS_INCREMENTAL} ${ZOOKEEPER_VERSION}
Original file line number Diff line number Diff line change
Expand Up @@ -153,6 +153,6 @@ function run_ha_test() {
STATE_BACKEND_TYPE=${1:-file}
STATE_BACKEND_FILE_ASYNC=${2:-true}
STATE_BACKEND_ROCKS_INCREMENTAL=${3:-false}
ZOOKEEPER_VERSION=${4:-3.4}
ZOOKEEPER_VERSION=${4:-3.5}

run_test_with_timeout 900 run_ha_test 4 ${STATE_BACKEND_TYPE} ${STATE_BACKEND_FILE_ASYNC} ${STATE_BACKEND_ROCKS_INCREMENTAL} ${ZOOKEEPER_VERSION}
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
import org.apache.flink.table.types.logical.LogicalTypeRoot;
import org.apache.flink.util.function.TriFunction;

import org.apache.flink.shaded.curator4.com.google.common.collect.ImmutableMap;
import org.apache.flink.shaded.curator5.com.google.common.collect.ImmutableMap;

import org.apache.hadoop.hive.common.type.HiveDecimal;
import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,8 @@

package org.apache.flink.runtime.checkpoint;

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.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.state.ConnectionState;

import javax.annotation.Nullable;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,8 +18,8 @@

package org.apache.flink.runtime.checkpoint;

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.curator5.org.apache.curator.framework.state.ConnectionState;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.state.ConnectionStateListener;

import java.util.Optional;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,10 +23,10 @@
import org.apache.flink.runtime.jobmanager.HighAvailabilityMode;
import org.apache.flink.runtime.util.ZooKeeperUtils;

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.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.shared.SharedCount;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.shared.VersionedValue;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.state.ConnectionState;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
import org.apache.flink.runtime.state.SharedStateRegistryFactory;
import org.apache.flink.runtime.util.ZooKeeperUtils;

import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFramework;

import java.util.concurrent.Executor;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,8 +28,8 @@
import org.apache.flink.runtime.jobmanager.JobGraphStore;
import org.apache.flink.runtime.util.ZooKeeperUtils;

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.curator5.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator5.org.apache.curator.utils.ZKPaths;
import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException;

import java.io.IOException;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,8 @@

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.api.UnhandledErrorListener;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.api.UnhandledErrorListener;

import java.io.Closeable;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@
import org.apache.flink.util.ExceptionUtils;
import org.apache.flink.util.FlinkRuntimeException;

import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFramework;

import javax.annotation.Nullable;
import javax.annotation.concurrent.GuardedBy;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,11 +20,11 @@

import org.apache.flink.api.common.JobID;

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.curator5.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.cache.PathChildrenCache;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.cache.PathChildrenCacheListener;
import org.apache.flink.shaded.curator5.org.apache.curator.utils.ZKPaths;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@
import org.apache.flink.util.AbstractID;
import org.apache.flink.util.Preconditions;

import org.apache.flink.shaded.curator4.com.google.common.collect.ImmutableList;
import org.apache.flink.shaded.curator5.com.google.common.collect.ImmutableList;

import java.util.ArrayList;
import java.util.Collections;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,13 +23,13 @@
import org.apache.flink.runtime.util.ZooKeeperUtils;
import org.apache.flink.util.ExceptionUtils;

import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
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.TreeCache;
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.curator5.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.cache.ChildData;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.cache.TreeCache;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatch;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.state.ConnectionState;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.state.ConnectionStateListener;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@

import org.apache.flink.runtime.rpc.FatalErrorHandler;

import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFramework;

/**
* {@link LeaderElectionDriverFactory} implementation for Zookeeper.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,14 +23,14 @@
import org.apache.flink.util.Preconditions;
import org.apache.flink.util.concurrent.Executors;

import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
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.TreeCache;
import org.apache.flink.shaded.curator4.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
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.curator5.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.cache.ChildData;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.cache.TreeCache;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.cache.TreeCacheSelector;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatch;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.leader.LeaderLatchListener;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.state.ConnectionState;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.state.ConnectionStateListener;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@

import org.apache.flink.util.Preconditions;

import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFramework;

/** Factory for {@link ZooKeeperMultipleComponentLeaderElectionDriver}. */
public class ZooKeeperMultipleComponentLeaderElectionDriverFactory
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,11 +25,11 @@
import org.apache.flink.runtime.util.ZooKeeperUtils;
import org.apache.flink.util.ExceptionUtils;

import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
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.TreeCache;
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.curator5.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.cache.ChildData;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.cache.TreeCache;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.state.ConnectionState;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.state.ConnectionStateListener;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@

import org.apache.flink.runtime.rpc.FatalErrorHandler;

import org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFramework;
import org.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFramework;

/** {@link LeaderRetrievalDriverFactory} implementation for Zookeeper. */
public class ZooKeeperLeaderRetrievalDriverFactory implements LeaderRetrievalDriverFactory {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@
import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
import org.apache.flink.util.Preconditions;

import org.apache.flink.shaded.curator4.com.google.common.collect.Iterables;
import org.apache.flink.shaded.curator5.com.google.common.collect.Iterables;

import javax.annotation.Nullable;

Expand Down
Loading

0 comments on commit 8ddfd59

Please sign in to comment.