Skip to content

Commit d13c1f6

Browse files
KAFKA-19480: Recreate /migration when it has null value (#20627)
When using the zookeeper-security-migration tool without the '–enable.path.check' option, the script not only updates the ACLs for the existing znodes, but also creates any non-existing ones (with the ACL options specified) using null values based on the list defined in `ZkData.SecureRootPaths`. This is especially problematic for the /migration znode as the current logic only checks for the existence of the znode and later the migration process will hang when it tries to parse the null value over and over again. In summary, the migration cannot be completed if the zookeeper-security-migration script was run previously, and the only workaround is to manually remove the /migration znode in such cases. I propose a simple fix to circumvent the manual step by recreating the /migration znode if it contains a null value. --------- Co-authored-by: Gergely Harmadas <[email protected]>
1 parent c29c130 commit d13c1f6

File tree

5 files changed

+74
-15
lines changed

5 files changed

+74
-15
lines changed

core/src/main/scala/kafka/admin/ZkSecurityMigrator.scala

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ import joptsimple.{OptionSet, OptionSpec, OptionSpecBuilder}
2121
import kafka.server.KafkaConfig
2222
import kafka.utils.{Exit, Logging, ToolsUtils}
2323
import kafka.utils.Implicits._
24-
import kafka.zk.{ControllerZNode, KafkaZkClient, ZkData, ZkSecurityMigratorUtils}
24+
import kafka.zk.{ControllerZNode, KafkaZkClient, MigrationZNode, ZkData, ZkSecurityMigratorUtils}
2525
import org.apache.kafka.common.security.JaasUtils
2626
import org.apache.kafka.common.utils.{Time, Utils}
2727
import org.apache.kafka.server.config.ZkConfigs
@@ -260,12 +260,18 @@ class ZkSecurityMigrator(zkClient: KafkaZkClient) extends Logging {
260260
}
261261

262262
private def run(enablePathCheck: Boolean): Unit = {
263+
def skipSetAcl(path: String): Boolean = {
264+
val isControllerPath = path == ControllerZNode.path
265+
val isMigrationPath = path == MigrationZNode.path
266+
(isControllerPath || isMigrationPath) && !zkClient.pathExists(path)
267+
}
268+
263269
try {
264270
setAclIndividually("/")
265271
checkPathExistenceAndMaybeExit(enablePathCheck)
266272
for (path <- ZkData.SecureRootPaths) {
267273
debug("Going to set ACL for %s".format(path))
268-
if (path == ControllerZNode.path && !zkClient.pathExists(path)) {
274+
if (skipSetAcl(path)) {
269275
debug("Ignoring to set ACL for %s, because it doesn't exist".format(path))
270276
} else {
271277
zkClient.makeSurePersistentPathExists(path)

core/src/main/scala/kafka/zk/KafkaZkClient.scala

Lines changed: 16 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -1734,20 +1734,32 @@ class KafkaZkClient private[zk] (
17341734
val getDataResponse = retryRequestUntilConnected(getDataRequest)
17351735
getDataResponse.resultCode match {
17361736
case Code.OK =>
1737-
MigrationZNode.decode(getDataResponse.data, getDataResponse.stat.getVersion, getDataResponse.stat.getMtime)
1737+
Option(getDataResponse.data) match {
1738+
case Some(data) =>
1739+
MigrationZNode.decode(data, getDataResponse.stat.getVersion, getDataResponse.stat.getMtime)
1740+
case None =>
1741+
info("Migration znode exists with null data, recreating initial migration state")
1742+
createInitialMigrationState(initialState, removeFirst = true)
1743+
}
17381744
case Code.NONODE =>
17391745
createInitialMigrationState(initialState)
17401746
case _ => throw getDataResponse.resultException.get
17411747
}
17421748
}
17431749

1744-
private def createInitialMigrationState(initialState: ZkMigrationLeadershipState): ZkMigrationLeadershipState = {
1745-
val createRequest = CreateRequest(
1750+
private def createInitialMigrationState(initialState: ZkMigrationLeadershipState, removeFirst: Boolean = false): ZkMigrationLeadershipState = {
1751+
val createOp = CreateOp(
17461752
MigrationZNode.path,
17471753
MigrationZNode.encode(initialState),
17481754
defaultAcls(MigrationZNode.path),
17491755
CreateMode.PERSISTENT)
1750-
val response = retryRequestUntilConnected(createRequest)
1756+
val deleteOp = DeleteOp(MigrationZNode.path, ZkVersion.MatchAnyVersion)
1757+
val multi = if (removeFirst) {
1758+
MultiRequest(Seq(deleteOp, createOp))
1759+
} else {
1760+
MultiRequest(Seq(createOp))
1761+
}
1762+
val response = retryRequestUntilConnected(multi)
17511763
response.maybeThrow()
17521764
initialState.withMigrationZkVersion(0)
17531765
}

core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala

Lines changed: 33 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,8 @@ import org.apache.kafka.common.security.auth.SecurityProtocol
3838
import org.apache.kafka.common.utils.Time
3939
import org.apache.kafka.server.common.MetadataVersion
4040
import org.apache.zookeeper.client.ZKClientConfig
41+
import org.junit.jupiter.params.ParameterizedTest
42+
import org.junit.jupiter.params.provider.ValueSource
4143

4244
import scala.jdk.CollectionConverters._
4345
import scala.collection.Seq
@@ -147,11 +149,12 @@ class ZkAuthorizationTest extends QuorumTestHarness with Logging {
147149
* Tests the migration tool when making an unsecure
148150
* cluster secure.
149151
*/
150-
@Test
151-
def testZkMigration(): Unit = {
152+
@ParameterizedTest
153+
@ValueSource(booleans = Array(false, true))
154+
def testZkMigration(includeAllZnodes: Boolean): Unit = {
152155
val unsecureZkClient = newKafkaZkClient(zkConnect, isSecure = false)
153156
try {
154-
testMigration(zkConnect, unsecureZkClient, zkClient)
157+
testMigration(zkConnect, unsecureZkClient, zkClient, includeAllZnodes)
155158
} finally {
156159
unsecureZkClient.close()
157160
}
@@ -161,11 +164,12 @@ class ZkAuthorizationTest extends QuorumTestHarness with Logging {
161164
* Tests the migration tool when making a secure
162165
* cluster unsecure.
163166
*/
164-
@Test
165-
def testZkAntiMigration(): Unit = {
167+
@ParameterizedTest
168+
@ValueSource(booleans = Array(false, true))
169+
def testZkAntiMigration(includeAllZnodes: Boolean): Unit = {
166170
val unsecureZkClient = newKafkaZkClient(zkConnect, isSecure = false)
167171
try {
168-
testMigration(zkConnect, zkClient, unsecureZkClient)
172+
testMigration(zkConnect, zkClient, unsecureZkClient, includeAllZnodes)
169173
} finally {
170174
unsecureZkClient.close()
171175
}
@@ -218,9 +222,17 @@ class ZkAuthorizationTest extends QuorumTestHarness with Logging {
218222
* Exercises the migration tool. It is used in these test cases:
219223
* testZkMigration, testZkAntiMigration, testChroot.
220224
*/
221-
private def testMigration(zkUrl: String, firstZk: KafkaZkClient, secondZk: KafkaZkClient): Unit = {
225+
private def testMigration(
226+
zkUrl: String,
227+
firstZk: KafkaZkClient,
228+
secondZk: KafkaZkClient,
229+
includeAllZnodes: Boolean = true): Unit = {
222230
info(s"zkConnect string: $zkUrl")
223-
for (path <- ZkData.SecureRootPaths ++ ZkData.SensitiveRootPaths) {
231+
// Optionally do not create controller and migration znodes
232+
val predicate: String => Boolean = if (includeAllZnodes) _ => true else skipCreateZnodes
233+
val paths = (ZkData.SecureRootPaths ++ ZkData.SensitiveRootPaths).filter(predicate)
234+
235+
for (path <- paths) {
224236
info(s"Creating $path")
225237
firstZk.makeSurePersistentPathExists(path)
226238
// Create a child for each znode to exercise the recurrent
@@ -241,7 +253,7 @@ class ZkAuthorizationTest extends QuorumTestHarness with Logging {
241253
}
242254
ZkSecurityMigrator.run(Array(s"--zookeeper.acl=$secureOpt", s"--zookeeper.connect=$zkUrl"))
243255
info("Done with migration")
244-
for (path <- ZkData.SecureRootPaths ++ ZkData.SensitiveRootPaths) {
256+
for (path <- paths) {
245257
val sensitive = ZkData.sensitivePath(path)
246258
val listParent = secondZk.getAcl(path)
247259
assertTrue(isAclCorrect(listParent, secondZk.secure, sensitive), path)
@@ -257,6 +269,18 @@ class ZkAuthorizationTest extends QuorumTestHarness with Logging {
257269
ZkData.sensitivePath(ExtendedAclZNode.path)), "/kafka-acl-extended")
258270
assertTrue(isAclCorrect(firstZk.getAcl("/feature"), secondZk.secure,
259271
ZkData.sensitivePath(FeatureZNode.path)), "ACL mismatch for /feature path")
272+
273+
if (!includeAllZnodes) {
274+
// Check controller and migration znodes should not be created
275+
assertFalse(firstZk.pathExists(ControllerZNode.path))
276+
assertFalse(firstZk.pathExists(MigrationZNode.path))
277+
}
278+
}
279+
280+
private def skipCreateZnodes(path: String): Boolean = {
281+
val isNotControllerPath = path != ControllerZNode.path
282+
val isNotMigrationPath = path != MigrationZNode.path
283+
isNotControllerPath && isNotMigrationPath
260284
}
261285

262286
/**

core/src/test/scala/unit/kafka/zk/KafkaZkClientTest.scala

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1442,6 +1442,21 @@ class KafkaZkClientTest extends QuorumTestHarness {
14421442
} finally System.clearProperty(ZKConfig.JUTE_MAXBUFFER)
14431443
}
14441444

1445+
@Test
1446+
def testMigrationZnodeWithNullValue(): Unit = {
1447+
val (controllerEpoch, stat) = zkClient.getControllerEpoch.get
1448+
var migrationState = new ZkMigrationLeadershipState(3000, 42, 100, 42, Time.SYSTEM.milliseconds(), -1, controllerEpoch, stat.getVersion)
1449+
zkClient.retryRequestUntilConnected(CreateRequest(
1450+
MigrationZNode.path,
1451+
null,
1452+
zkClient.defaultAcls(MigrationZNode.path),
1453+
CreateMode.PERSISTENT))
1454+
1455+
migrationState = zkClient.getOrCreateMigrationState(migrationState)
1456+
1457+
assertEquals(0, migrationState.migrationZkVersion())
1458+
}
1459+
14451460
@Test
14461461
def testFailToUpdateMigrationZNode(): Unit = {
14471462
val (controllerEpoch, stat) = zkClient.getControllerEpoch.get

docs/ops.html

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -4064,6 +4064,8 @@ <h3>Provisioning the KRaft controller quorum</h3>
40644064

40654065
<p>The new standalone controller in the example configuration above should be formatted using the <code>kafka-storage format --standalone</code>command.</p>
40664066

4067+
<p>Note: The migration can stall if the <a href="#zk_authz_migration">ZooKeeper Security Migration Tool</a> was previously executed (fixed from 3.9.2, see <a href="https://issues.apache.org/jira/browse/KAFKA-19480">KAFKA-19026</a> for more details). As a workaround, the malformed "/migration" node can be removed from ZooKeeper by running <code>delete /migration</code> with the <code>zookeeper-shell.sh</code> CLI tool.</p>
4068+
40674069
<p><em>Note: The KRaft cluster <code>node.id</code> values must be different from any existing ZK broker <code>broker.id</code>.
40684070
In KRaft-mode, the brokers and controllers share the same Node ID namespace.</em></p>
40694071

0 commit comments

Comments
 (0)