mirror of https://github.com/apache/kafka.git
KAFKA-7987: Reinitialize ZookeeperClient after auth failures (#7751)
Schedules client reinitialization if Zookeeper auth failure is encountered. This allows for reconnections when transient network errors are encountered during connection establishment. The Zookeeper client doesn't expose details of the auth failure so we can't determine whether an error is retriable or not, so all auth failures are retried. Co-authored-by: Rajini Sivaram <rajinisivaram@googlemail.com> Reviewers: Jun Rao <junrao@gmail.com>
This commit is contained in:
parent
27153e2268
commit
5df8457e05
|
|
@ -26,6 +26,7 @@ import com.yammer.metrics.core.MetricName
|
|||
import kafka.metrics.KafkaMetricsGroup
|
||||
import kafka.utils.CoreUtils.{inLock, inReadLock, inWriteLock}
|
||||
import kafka.utils.{KafkaScheduler, Logging}
|
||||
import kafka.zookeeper.ZooKeeperClient._
|
||||
import org.apache.kafka.common.utils.Time
|
||||
import org.apache.zookeeper.AsyncCallback.{Children2Callback, DataCallback, StatCallback}
|
||||
import org.apache.zookeeper.KeeperException.Code
|
||||
|
|
@ -39,6 +40,10 @@ import scala.jdk.CollectionConverters._
|
|||
import scala.collection.Seq
|
||||
import scala.collection.mutable.Set
|
||||
|
||||
object ZooKeeperClient {
|
||||
val RetryBackoffMs = 1000
|
||||
}
|
||||
|
||||
/**
|
||||
* A ZooKeeper client that encourages pipelined requests.
|
||||
*
|
||||
|
|
@ -81,7 +86,8 @@ class ZooKeeperClient(connectString: String,
|
|||
private val zNodeChildChangeHandlers = new ConcurrentHashMap[String, ZNodeChildChangeHandler]().asScala
|
||||
private val inFlightRequests = new Semaphore(maxInFlightRequests)
|
||||
private val stateChangeHandlers = new ConcurrentHashMap[String, StateChangeHandler]().asScala
|
||||
private[zookeeper] val expiryScheduler = new KafkaScheduler(threads = 1, "zk-session-expiry-handler")
|
||||
private[zookeeper] val reinitializeScheduler = new KafkaScheduler(threads = 1, s"zk-client-${threadPrefix}reinit-")
|
||||
private var isFirstConnectionEstablished = false
|
||||
|
||||
private val metricNames = Set[String]()
|
||||
|
||||
|
|
@ -115,7 +121,7 @@ class ZooKeeperClient(connectString: String,
|
|||
|
||||
metricNames += "SessionState"
|
||||
|
||||
expiryScheduler.startup()
|
||||
reinitializeScheduler.startup()
|
||||
try waitUntilConnected(connectionTimeoutMs, TimeUnit.MILLISECONDS)
|
||||
catch {
|
||||
case e: Throwable =>
|
||||
|
|
@ -269,6 +275,7 @@ class ZooKeeperClient(connectString: String,
|
|||
} else if (state == States.CLOSED) {
|
||||
throw new ZooKeeperClientExpiredException("Session expired either before or while waiting for connection")
|
||||
}
|
||||
isFirstConnectionEstablished = true
|
||||
}
|
||||
info("Connected.")
|
||||
}
|
||||
|
|
@ -344,7 +351,7 @@ class ZooKeeperClient(connectString: String,
|
|||
// Shutdown scheduler outside of lock to avoid deadlock if scheduler
|
||||
// is waiting for lock to process session expiry. Close expiry thread
|
||||
// first to ensure that new clients are not created during close().
|
||||
expiryScheduler.shutdown()
|
||||
reinitializeScheduler.shutdown()
|
||||
|
||||
inWriteLock(initializationLock) {
|
||||
zNodeChangeHandlers.clear()
|
||||
|
|
@ -383,7 +390,7 @@ class ZooKeeperClient(connectString: String,
|
|||
} catch {
|
||||
case e: Exception =>
|
||||
info("Error when recreating ZooKeeper, retrying after a short sleep", e)
|
||||
Thread.sleep(1000)
|
||||
Thread.sleep(RetryBackoffMs)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -419,13 +426,15 @@ class ZooKeeperClient(connectString: String,
|
|||
}
|
||||
|
||||
// Visibility for testing
|
||||
private[zookeeper] def scheduleSessionExpiryHandler(): Unit = {
|
||||
expiryScheduler.scheduleOnce("zk-session-expired", () => {
|
||||
info("Session expired.")
|
||||
private[zookeeper] def scheduleReinitialize(name: String, message: String, delayMs: Long): Unit = {
|
||||
reinitializeScheduler.schedule(name, () => {
|
||||
info(message)
|
||||
reinitialize()
|
||||
})
|
||||
}, delayMs, period = -1L, unit = TimeUnit.MILLISECONDS)
|
||||
}
|
||||
|
||||
private def threadPrefix: String = name.map(n => n.replaceAll("\\s", "") + "-").getOrElse("")
|
||||
|
||||
// package level visibility for testing only
|
||||
private[zookeeper] object ZooKeeperClientWatcher extends Watcher {
|
||||
override def process(event: WatchedEvent): Unit = {
|
||||
|
|
@ -440,8 +449,15 @@ class ZooKeeperClient(connectString: String,
|
|||
if (state == KeeperState.AuthFailed) {
|
||||
error("Auth failed.")
|
||||
stateChangeHandlers.values.foreach(_.onAuthFailure())
|
||||
|
||||
// If this is during initial startup, we fail fast. Otherwise, schedule retry.
|
||||
val initialized = inLock(isConnectedOrExpiredLock) {
|
||||
isFirstConnectionEstablished
|
||||
}
|
||||
if (initialized)
|
||||
scheduleReinitialize("auth-failed", "Reinitializing due to auth failure.", RetryBackoffMs)
|
||||
} else if (state == KeeperState.Expired) {
|
||||
scheduleSessionExpiryHandler()
|
||||
scheduleReinitialize("session-expired", "Session expired.", delayMs = 0L)
|
||||
}
|
||||
case Some(path) =>
|
||||
(event.getType: @unchecked) match {
|
||||
|
|
|
|||
|
|
@ -0,0 +1,186 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
||||
package kafka.security.authorizer
|
||||
|
||||
import java.net.InetAddress
|
||||
import java.util
|
||||
import java.util.UUID
|
||||
import java.util.concurrent.{Executors, TimeUnit}
|
||||
|
||||
import javax.security.auth.Subject
|
||||
import javax.security.auth.callback.CallbackHandler
|
||||
import kafka.api.SaslSetup
|
||||
import kafka.security.authorizer.AclEntry.WildcardHost
|
||||
import kafka.server.KafkaConfig
|
||||
import kafka.utils.JaasTestUtils.{JaasModule, JaasSection}
|
||||
import kafka.utils.{JaasTestUtils, TestUtils}
|
||||
import kafka.zk.{KafkaZkClient, ZooKeeperTestHarness}
|
||||
import kafka.zookeeper.ZooKeeperClient
|
||||
import org.apache.kafka.common.acl.{AccessControlEntry, AccessControlEntryFilter, AclBinding, AclBindingFilter}
|
||||
import org.apache.kafka.common.acl.AclOperation.{READ, WRITE}
|
||||
import org.apache.kafka.common.acl.AclPermissionType.ALLOW
|
||||
import org.apache.kafka.common.network.{ClientInformation, ListenerName}
|
||||
import org.apache.kafka.common.protocol.ApiKeys
|
||||
import org.apache.kafka.common.requests.{RequestContext, RequestHeader}
|
||||
import org.apache.kafka.common.resource.PatternType.LITERAL
|
||||
import org.apache.kafka.common.resource.ResourcePattern
|
||||
import org.apache.kafka.common.resource.ResourceType.TOPIC
|
||||
import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
|
||||
import org.apache.kafka.test.{TestUtils => JTestUtils}
|
||||
import org.apache.zookeeper.server.auth.DigestLoginModule
|
||||
import org.junit.Assert.assertEquals
|
||||
import org.junit.{After, Before, Test}
|
||||
|
||||
import scala.jdk.CollectionConverters._
|
||||
import scala.collection.Seq
|
||||
|
||||
class AclAuthorizerWithZkSaslTest extends ZooKeeperTestHarness with SaslSetup {
|
||||
|
||||
private val aclAuthorizer = new AclAuthorizer
|
||||
private val aclAuthorizer2 = new AclAuthorizer
|
||||
private val resource: ResourcePattern = new ResourcePattern(TOPIC, "foo-" + UUID.randomUUID(), LITERAL)
|
||||
private val username = "alice"
|
||||
private val principal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username)
|
||||
private val requestContext = newRequestContext(principal, InetAddress.getByName("192.168.0.1"))
|
||||
private val executor = Executors.newSingleThreadScheduledExecutor
|
||||
private var config: KafkaConfig = _
|
||||
|
||||
@Before
|
||||
override def setUp(): Unit = {
|
||||
// Allow failed clients to avoid server closing the connection before reporting AuthFailed.
|
||||
System.setProperty("zookeeper.allowSaslFailedClients", "true")
|
||||
|
||||
// Configure ZK SASL with TestableDigestLoginModule for clients to inject failures
|
||||
TestableDigestLoginModule.reset()
|
||||
val jaasSections = JaasTestUtils.zkSections
|
||||
val serverJaas = jaasSections.filter(_.contextName == "Server")
|
||||
val clientJaas = jaasSections.filter(_.contextName == "Client")
|
||||
.map(section => new TestableJaasSection(section.contextName, section.modules))
|
||||
startSasl(serverJaas ++ clientJaas)
|
||||
|
||||
// Increase maxUpdateRetries to avoid transient failures
|
||||
aclAuthorizer.maxUpdateRetries = Int.MaxValue
|
||||
aclAuthorizer2.maxUpdateRetries = Int.MaxValue
|
||||
|
||||
super.setUp()
|
||||
config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))
|
||||
|
||||
aclAuthorizer.configure(config.originals)
|
||||
aclAuthorizer2.configure(config.originals)
|
||||
}
|
||||
|
||||
@After
|
||||
override def tearDown(): Unit = {
|
||||
System.clearProperty("zookeeper.allowSaslFailedClients")
|
||||
TestableDigestLoginModule.reset()
|
||||
executor.shutdownNow()
|
||||
aclAuthorizer.close()
|
||||
aclAuthorizer2.close()
|
||||
super.tearDown()
|
||||
}
|
||||
|
||||
@Test
|
||||
def testAclUpdateWithSessionExpiration(): Unit = {
|
||||
zkClient(aclAuthorizer).currentZooKeeper.getTestable.injectSessionExpiration()
|
||||
zkClient(aclAuthorizer2).currentZooKeeper.getTestable.injectSessionExpiration()
|
||||
verifyAclUpdate()
|
||||
}
|
||||
|
||||
@Test
|
||||
def testAclUpdateWithAuthFailure(): Unit = {
|
||||
injectTransientAuthenticationFailure()
|
||||
verifyAclUpdate()
|
||||
}
|
||||
|
||||
private def injectTransientAuthenticationFailure(): Unit = {
|
||||
TestableDigestLoginModule.injectInvalidCredentials()
|
||||
zkClient(aclAuthorizer).currentZooKeeper.getTestable.injectSessionExpiration()
|
||||
zkClient(aclAuthorizer2).currentZooKeeper.getTestable.injectSessionExpiration()
|
||||
executor.schedule((() => TestableDigestLoginModule.reset()): Runnable,
|
||||
ZooKeeperClient.RetryBackoffMs * 2, TimeUnit.MILLISECONDS)
|
||||
}
|
||||
|
||||
private def verifyAclUpdate(): Unit = {
|
||||
val allowReadAcl = new AccessControlEntry(principal.toString, WildcardHost, READ, ALLOW)
|
||||
val allowWriteAcl = new AccessControlEntry(principal.toString, WildcardHost, WRITE, ALLOW)
|
||||
val acls = Set(allowReadAcl, allowWriteAcl)
|
||||
|
||||
TestUtils.retry(maxWaitMs = 15000) {
|
||||
try {
|
||||
addAcls(aclAuthorizer, acls, resource)
|
||||
} catch {
|
||||
case _: Exception => // Ignore error and retry
|
||||
}
|
||||
assertEquals(acls, getAcls(aclAuthorizer, resource))
|
||||
}
|
||||
val (acls2, _) = TestUtils.computeUntilTrue(getAcls(aclAuthorizer2, resource)) { _ == acls }
|
||||
assertEquals(acls, acls2)
|
||||
}
|
||||
|
||||
private def zkClient(authorizer: AclAuthorizer): KafkaZkClient = {
|
||||
JTestUtils.fieldValue(authorizer, classOf[AclAuthorizer], "zkClient")
|
||||
}
|
||||
|
||||
private def addAcls(authorizer: AclAuthorizer, aces: Set[AccessControlEntry], resourcePattern: ResourcePattern): Unit = {
|
||||
val bindings = aces.map { ace => new AclBinding(resourcePattern, ace) }
|
||||
authorizer.createAcls(requestContext, bindings.toList.asJava).asScala
|
||||
.map(_.toCompletableFuture.get)
|
||||
.foreach { result => result.exception.ifPresent { e => throw e } }
|
||||
}
|
||||
|
||||
private def getAcls(authorizer: AclAuthorizer, resourcePattern: ResourcePattern): Set[AccessControlEntry] = {
|
||||
val acls = authorizer.acls(new AclBindingFilter(resourcePattern.toFilter, AccessControlEntryFilter.ANY)).asScala.toSet
|
||||
acls.map(_.entry)
|
||||
}
|
||||
|
||||
private def newRequestContext(principal: KafkaPrincipal, clientAddress: InetAddress, apiKey: ApiKeys = ApiKeys.PRODUCE): RequestContext = {
|
||||
val securityProtocol = SecurityProtocol.SASL_PLAINTEXT
|
||||
val header = new RequestHeader(apiKey, 2, "", 1) //ApiKeys apiKey, short version, String clientId, int correlation
|
||||
new RequestContext(header, "", clientAddress, principal, ListenerName.forSecurityProtocol(securityProtocol),
|
||||
securityProtocol, ClientInformation.EMPTY)
|
||||
}
|
||||
}
|
||||
|
||||
object TestableDigestLoginModule {
|
||||
@volatile var injectedPassword: Option[String] = None
|
||||
|
||||
def reset(): Unit = {
|
||||
injectedPassword = None
|
||||
}
|
||||
|
||||
def injectInvalidCredentials(): Unit = {
|
||||
injectedPassword = Some("invalidPassword")
|
||||
}
|
||||
}
|
||||
|
||||
class TestableDigestLoginModule extends DigestLoginModule {
|
||||
override def initialize(subject: Subject, callbackHandler: CallbackHandler, sharedState: util.Map[String, _], options: util.Map[String, _]): Unit = {
|
||||
super.initialize(subject, callbackHandler, sharedState, options)
|
||||
val injectedPassword = TestableDigestLoginModule.injectedPassword
|
||||
injectedPassword.foreach { newPassword =>
|
||||
val oldPassword = subject.getPrivateCredentials.asScala.head
|
||||
subject.getPrivateCredentials.add(newPassword)
|
||||
subject.getPrivateCredentials.remove(oldPassword)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class TestableJaasSection(contextName: String, modules: Seq[JaasModule]) extends JaasSection(contextName, modules) {
|
||||
override def toString: String = {
|
||||
super.toString.replaceFirst(classOf[DigestLoginModule].getName, classOf[TestableDigestLoginModule].getName)
|
||||
}
|
||||
}
|
||||
|
|
@ -585,7 +585,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
|
|||
}
|
||||
private def verifyHandlerThread(): Unit = {
|
||||
val threadName = Thread.currentThread.getName
|
||||
assertTrue(s"Unexpected thread + $threadName", threadName.startsWith(zooKeeperClient.expiryScheduler.threadNamePrefix))
|
||||
assertTrue(s"Unexpected thread + $threadName", threadName.startsWith(zooKeeperClient.reinitializeScheduler.threadNamePrefix))
|
||||
}
|
||||
})
|
||||
|
||||
|
|
@ -625,7 +625,7 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
|
|||
zooKeeperClient.close()
|
||||
responseExecutor.shutdownNow()
|
||||
}
|
||||
assertFalse("Expiry executor not shutdown", zooKeeperClient.expiryScheduler.isStarted)
|
||||
assertFalse("Expiry executor not shutdown", zooKeeperClient.reinitializeScheduler.isStarted)
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
@ -633,9 +633,9 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
|
|||
val semaphore = new Semaphore(0)
|
||||
val closeExecutor = Executors.newSingleThreadExecutor
|
||||
try {
|
||||
zooKeeperClient.expiryScheduler.schedule("test", () => semaphore.acquireUninterruptibly(),
|
||||
zooKeeperClient.reinitializeScheduler.schedule("test", () => semaphore.acquireUninterruptibly(),
|
||||
delay = 0, period = -1, TimeUnit.SECONDS)
|
||||
zooKeeperClient.scheduleSessionExpiryHandler()
|
||||
zooKeeperClient.scheduleReinitialize("session-expired", "Session expired.", delayMs = 0L)
|
||||
val closeFuture = closeExecutor.submit(new Runnable {
|
||||
override def run(): Unit = {
|
||||
zooKeeperClient.close()
|
||||
|
|
@ -645,12 +645,31 @@ class ZooKeeperClientTest extends ZooKeeperTestHarness {
|
|||
assertTrue(zooKeeperClient.currentZooKeeper.getState.isAlive) // Client should be closed after expiry handler
|
||||
semaphore.release()
|
||||
closeFuture.get(10, TimeUnit.SECONDS)
|
||||
assertFalse("Expiry executor not shutdown", zooKeeperClient.expiryScheduler.isStarted)
|
||||
assertFalse("Expiry executor not shutdown", zooKeeperClient.reinitializeScheduler.isStarted)
|
||||
} finally {
|
||||
closeExecutor.shutdownNow()
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testReinitializeAfterAuthFailure(): Unit = {
|
||||
val sessionInitializedCountDownLatch = new CountDownLatch(1)
|
||||
val changeHandler = new StateChangeHandler {
|
||||
override val name = this.getClass.getName
|
||||
override def beforeInitializingSession(): Unit = {
|
||||
sessionInitializedCountDownLatch.countDown()
|
||||
}
|
||||
}
|
||||
|
||||
zooKeeperClient.close()
|
||||
zooKeeperClient = new ZooKeeperClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, Int.MaxValue, time,
|
||||
"testMetricGroup", "testMetricType")
|
||||
zooKeeperClient.registerStateChangeHandler(changeHandler)
|
||||
|
||||
zooKeeperClient.ZooKeeperClientWatcher.process(new WatchedEvent(EventType.None, KeeperState.AuthFailed, null))
|
||||
assertTrue("Failed to receive session initializing notification", sessionInitializedCountDownLatch.await(5, TimeUnit.SECONDS))
|
||||
}
|
||||
|
||||
def isExpectedMetricName(metricName: MetricName, name: String): Boolean =
|
||||
metricName.getName == name && metricName.getGroup == "testMetricGroup" && metricName.getType == "testMetricType"
|
||||
|
||||
|
|
|
|||
Loading…
Reference in New Issue