Hi Filip,

Curator 4.2 is compatible with ZK 3.4 series. When it detects that ZK 3.4 
client is used, it will operate in ZK 3.4 compatible mode and won't call 
org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.server.quorum.flexible.QuorumMaj.<init>(Ljava/util/Map;).

Curator detects whether the ZK client is of 3.4 series by detecting the 
existence of 
org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.admin.ZooKeeperAdmin in 
classpath. (see 
org.apache.flink.shaded.curator4.org.apache.curator.utils.Compatibility). You 
can try searching for that class in your application's classpath to see if a 
higher version of ZK client is mistakenly packaged.

Hope that helps.

Best,
Zhanghao Chen
________________________________
From: Filip Karnicki <filip.karni...@gmail.com>
Sent: Monday, March 7, 2022 16:49
To: Zhanghao Chen <zhanghao.c...@outlook.com>
Cc: user <user@flink.apache.org>
Subject: Re: Shaded zookeeper - curator mismatch?

Hi  Zhanghao

it's 3.5.5

Thank you
Fil

On Sat, 5 Mar 2022 at 08:12, Zhanghao Chen 
<zhanghao.c...@outlook.com<mailto:zhanghao.c...@outlook.com>> wrote:
Hi Filip,

Could you share the version of the ZK server you are connecting to?


Best,
Zhanghao Chen
________________________________
From: Filip Karnicki <filip.karni...@gmail.com<mailto:filip.karni...@gmail.com>>
Sent: Friday, March 4, 2022 23:12
To: user <user@flink.apache.org<mailto:user@flink.apache.org>>
Subject: Shaded zookeeper - curator mismatch?

Hi, I believe there's a mismatch in shaded zookeeper/curator dependencies. I 
see that curator 4.2.0 needs zookeeper 3.5.4-beta, but it's still used in 
flink-shaded-zookeeper-34, which as far as I can tell is used by flink runtime 
1.14.3

https://mvnrepository.com/artifact/org.apache.flink/flink-runtime/1.14.3

https://github.com/apache/flink-shaded/blob/release-14.0/flink-shaded-zookeeper-parent/flink-shaded-zookeeper-34/pom.xml

I believe this might be causing an issue for us while running a statefun uber 
jar in a cloudera cluster (indeed QuorumMaj only has a constructor that takes 
an int in zk 3.4.x, not a map)

Am I understanding this correctly? Is this something that we need to fix with a 
patch?



org.apache.flink.runtime.entrypoint.ClusterEntrypointException: Failed to 
initialize the cluster entrypoint YarnJobClusterEntrypoint.

        at 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.startCluster(ClusterEntrypoint.java:216)
 ~[fatty.fil.jar:?]

        at 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.runClusterEntrypoint(ClusterEntrypoint.java:617)
 [fatty.fil.jar:?]

        at 
org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint.main(YarnJobClusterEntrypoint.java:99)
 [flink-dist_2.12-1.14.0-csa1.6.1.0.jar:1.14.0-csa1.6.1.0]

Caused by: java.lang.NoSuchMethodError: 
org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.server.quorum.flexible.QuorumMaj.<init>(Ljava/util/Map;)V

        at 
org.apache.flink.shaded.curator4.org.apache.curator.framework.imps.EnsembleTracker.<init>(EnsembleTracker.java:57)
 ~[fatty.fil.jar:?]

        at 
org.apache.flink.shaded.curator4.org.apache.curator.framework.imps.CuratorFrameworkImpl.<init>(CuratorFrameworkImpl.java:159)
 ~[fatty.fil.jar:?]

        at 
org.apache.flink.shaded.curator4.org.apache.curator.framework.CuratorFrameworkFactory$Builder.build(CuratorFrameworkFactory.java:165)
 ~[fatty.fil.jar:?]

        at 
org.apache.flink.runtime.util.ZooKeeperUtils.startCuratorFramework(ZooKeeperUtils.java:248)
 ~[fatty.fil.jar:?]

        at 
org.apache.flink.runtime.util.ZooKeeperUtils.startCuratorFramework(ZooKeeperUtils.java:233)
 ~[fatty.fil.jar:?]

        at 
org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils.createHighAvailabilityServices(HighAvailabilityServicesUtils.java:124)
 ~[fatty.fil.jar:?]

        at 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.createHaServices(ClusterEntrypoint.java:361)
 ~[fatty.fil.jar:?]

        at 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.initializeServices(ClusterEntrypoint.java:318)
 ~[fatty.fil.jar:?]

        at 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.runCluster(ClusterEntrypoint.java:243)
 ~[fatty.fil.jar:?]

        at 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.lambda$startCluster$1(ClusterEntrypoint.java:193)
 ~[fatty.fil.jar:?]

        at java.security.AccessController.doPrivileged(Native Method) 
~[?:1.8.0_212]

        at javax.security.auth.Subject.doAs(Subject.java:422) ~[?:1.8.0_212]

        at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1898)
 ~[hadoop-common-3.1.1.7.1.7.74-6.jar:?]

        at 
org.apache.flink.runtime.security.contexts.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41)
 ~[fatty.fil.jar:?]

        at 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint.startCluster(ClusterEntrypoint.java:190)
 ~[fatty.fil.jar:?]

        ... 2 more



Many thanks

Fil

Reply via email to