From 9ccb3f3f1cf9a4105d28811f0af9848aa0462f1d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=91=A8=E4=B8=80=E5=B8=86?= <88070094+zhouyifan279@users.noreply.github.com> Date: Fri, 12 Nov 2021 20:26:30 +0800 Subject: [PATCH] [KYUUBI #1361][BACKPORT] [KYUUBI #1176] InvalidACL appears in the engine when zookeeper acl is turned on --- docs/deployment/settings.md | 1 + .../kyuubi/ha/HighAvailabilityConf.scala | 7 ++++ .../ha/client/ZooKeeperACLProvider.scala | 11 +++++- .../ha/client/ServiceDiscoverySuite.scala | 36 +++++++++++++------ 4 files changed, 44 insertions(+), 11 deletions(-) diff --git a/docs/deployment/settings.md b/docs/deployment/settings.md index a45e7fd32..3c8fba1c7 100644 --- a/docs/deployment/settings.md +++ b/docs/deployment/settings.md @@ -193,6 +193,7 @@ Key | Default | Meaning | Type | Since --- | --- | --- | --- | --- kyuubi\.ha\.engine\.ref
\.id|
<undefined>
|
The engine reference id will be attached to zookeeper node when engine started, and the kyuubi server will check it cyclically.
|
string
|
1.3.2
kyuubi\.ha\.zookeeper
\.acl\.enabled|
false
|
Set to true if the zookeeper ensemble is kerberized
|
boolean
|
1.0.0
+kyuubi\.ha\.zookeeper
\.acl\.engine\.enabled|
false
|
Set to true if the zookeeper ensemble is kerberized at engine side.
|
boolean
|
1.3.2
kyuubi\.ha\.zookeeper
\.connection\.base\.retry
\.wait|
1000
|
Initial amount of time to wait between retries to the zookeeper ensemble
|
int
|
1.0.0
kyuubi\.ha\.zookeeper
\.connection\.max
\.retries|
3
|
Max retry times for connecting to the zookeeper ensemble
|
int
|
1.0.0
kyuubi\.ha\.zookeeper
\.connection\.max\.retry
\.wait|
30000
|
Max amount of time to wait between retries for BOUNDED_EXPONENTIAL_BACKOFF policy can reach, or max time until elapsed for UNTIL_ELAPSED policy to connect the zookeeper ensemble
|
int
|
1.0.0
diff --git a/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/HighAvailabilityConf.scala b/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/HighAvailabilityConf.scala index d46539421..532856ae3 100644 --- a/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/HighAvailabilityConf.scala +++ b/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/HighAvailabilityConf.scala @@ -48,6 +48,13 @@ object HighAvailabilityConf { .booleanConf .createWithDefault(UserGroupInformation.isSecurityEnabled) + val HA_ZK_ACL_ENGINE_ENABLED: ConfigEntry[Boolean] = + buildConf("ha.zookeeper.acl.engine.enabled") + .doc("Set to true if the zookeeper ensemble is kerberized at engine side.") + .version("1.3.2") + .booleanConf + .createWithDefault(false) + val HA_ZK_CONN_MAX_RETRIES: ConfigEntry[Int] = buildConf("ha.zookeeper.connection.max.retries") .doc("Max retry times for connecting to the zookeeper ensemble") diff --git a/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/client/ZooKeeperACLProvider.scala b/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/client/ZooKeeperACLProvider.scala index 582f98206..923169095 100644 --- a/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/client/ZooKeeperACLProvider.scala +++ b/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/client/ZooKeeperACLProvider.scala @@ -33,11 +33,20 @@ class ZooKeeperACLProvider(conf: KyuubiConf) extends ACLProvider { */ override lazy val getDefaultAcl: java.util.List[ACL] = { val nodeAcls = new java.util.ArrayList[ACL] - if (conf.get(HighAvailabilityConf.HA_ZK_ACL_ENABLED)) { + + def addACL(): Unit = { // Read all to the world nodeAcls.addAll(ZooDefs.Ids.READ_ACL_UNSAFE) // Create/Delete/Write/Admin to the authenticated user nodeAcls.addAll(ZooDefs.Ids.CREATOR_ALL_ACL) + } + + if (conf.get(HighAvailabilityConf.HA_ZK_ACL_ENABLED) && + conf.get(HighAvailabilityConf.HA_ZK_ENGINE_REF_ID).isEmpty) { + addACL() + } else if (conf.get(HighAvailabilityConf.HA_ZK_ACL_ENGINE_ENABLED) && + conf.get(HighAvailabilityConf.HA_ZK_ENGINE_REF_ID).nonEmpty) { + addACL() } else { // ACLs for znodes on a non-kerberized cluster // Create/Read/Delete/Write/Admin to the world diff --git a/kyuubi-ha/src/test/scala/org/apache/kyuubi/ha/client/ServiceDiscoverySuite.scala b/kyuubi-ha/src/test/scala/org/apache/kyuubi/ha/client/ServiceDiscoverySuite.scala index 2217e80a5..468bdb592 100644 --- a/kyuubi-ha/src/test/scala/org/apache/kyuubi/ha/client/ServiceDiscoverySuite.scala +++ b/kyuubi-ha/src/test/scala/org/apache/kyuubi/ha/client/ServiceDiscoverySuite.scala @@ -19,12 +19,14 @@ package org.apache.kyuubi.ha.client import java.io.{File, IOException} import java.net.InetAddress +import java.util import javax.security.auth.login.Configuration import scala.collection.JavaConverters._ import org.apache.hadoop.util.StringUtils import org.apache.zookeeper.ZooDefs +import org.apache.zookeeper.data.ACL import org.scalatest.time.SpanSugar._ import org.apache.kyuubi.{KerberizedTestHelper, KYUUBI_VERSION} @@ -97,17 +99,31 @@ class ServiceDiscoverySuite extends KerberizedTestHelper { } test("acl for zookeeper") { - val provider = new ZooKeeperACLProvider(conf) - val acl = provider.getDefaultAcl - assert(acl.size() === 1) - assert(acl === ZooDefs.Ids.OPEN_ACL_UNSAFE) + val expectedNoACL = new util.ArrayList[ACL](ZooDefs.Ids.OPEN_ACL_UNSAFE) + val expectedEnableACL = new util.ArrayList[ACL](ZooDefs.Ids.READ_ACL_UNSAFE) + expectedEnableACL.addAll(ZooDefs.Ids.CREATOR_ALL_ACL) - val conf1 = conf.clone.set(HA_ZK_ACL_ENABLED, true) - val acl1 = new ZooKeeperACLProvider(conf1).getDefaultAcl - assert(acl1.size() === 2) - val expected = ZooDefs.Ids.READ_ACL_UNSAFE - expected.addAll(ZooDefs.Ids.CREATOR_ALL_ACL) - assert(acl1 === expected) + def assertACL(expected: util.List[ACL], actual: util.List[ACL]): Unit = { + assert(actual.size() == expected.size()) + assert(actual === expected) + } + + val acl = new ZooKeeperACLProvider(conf).getDefaultAcl + assertACL(expectedNoACL, acl) + + val serverConf = conf.clone.set(HA_ZK_ACL_ENABLED, true) + val serverACL = new ZooKeeperACLProvider(serverConf).getDefaultAcl + assertACL(expectedEnableACL, serverACL) + + val engineConf = serverConf.clone.set(HA_ZK_ENGINE_REF_ID, "ref") + engineConf.set(HA_ZK_ACL_ENGINE_ENABLED, false) + val engineACL = new ZooKeeperACLProvider(engineConf).getDefaultAcl + assertACL(expectedNoACL, engineACL) + + val enableEngineACLConf = serverConf.clone.set(HA_ZK_ENGINE_REF_ID, "ref") + enableEngineACLConf.set(HA_ZK_ACL_ENGINE_ENABLED, true) + val enableEngineACL = new ZooKeeperACLProvider(enableEngineACLConf).getDefaultAcl + assertACL(expectedEnableACL, enableEngineACL) } test("set up zookeeper auth") {