[ 
https://issues.apache.org/jira/browse/ZOOKEEPER-4885?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17897903#comment-17897903
 ] 

Xin Chen commented on ZOOKEEPER-4885:
-------------------------------------

LoginException was thrown through each login, about code:
{code:java}
if (ZooKeeperSaslClient.isEnabled()) {
    try {
        if (zooKeeperSaslClient != null) {
            zooKeeperSaslClient.shutdown();
        }
        zooKeeperSaslClient = new 
ZooKeeperSaslClient(SaslServerPrincipal.getServerPrincipal(addr));
    } catch (LoginException e) {
        // An authentication error occurred when the SASL client tried to 
initialize:
        // for Kerberos this means that the client failed to authenticate with 
the KDC.
        // This is different from an authentication error that occurs during 
communication
        // with the Zookeeper server, which is handled below.
        LOG.warn("SASL configuration failed: " + e + " Will continue connection 
to Zookeeper server without "
          + "SASL authentication, if Zookeeper server allows it.");
        eventThread.queueEvent(new WatchedEvent(
          Watcher.Event.EventType.None,
          Watcher.Event.KeeperState.AuthFailed, null));
        saslLoginFailed = true;
    }
} {code}
Clients enter AuthFailed state from the following code:
{code:java}
if (zooKeeperSaslClient != null) {
    boolean sendAuthEvent = false;
    if (zooKeeperSaslClient.getSaslState() == 
ZooKeeperSaslClient.SaslState.INITIAL) {
        try {
            zooKeeperSaslClient.initialize(ClientCnxn.this);
        } catch (SaslException e) {
           LOG.error("SASL authentication with Zookeeper Quorum member failed: 
" + e);
            state = States.AUTH_FAILED;
            sendAuthEvent = true;
        }
    }
    KeeperState authState = zooKeeperSaslClient.getKeeperState();
    if (authState != null) {
        if (authState == KeeperState.AuthFailed) {
            // An authentication error occurred during authentication with the 
Zookeeper Server.
            state = States.AUTH_FAILED;
            sendAuthEvent = true;
        } else {
            if (authState == KeeperState.SaslAuthenticated) {
                sendAuthEvent = true;
            }
        }
    }

    if (sendAuthEvent == true) {
        eventThread.queueEvent(new WatchedEvent(
              Watcher.Event.EventType.None,
              authState,null));
        if (state == States.AUTH_FAILED) {
            eventThread.queueEventOfDeath();
        }
    }
}{code}
     I want to know if the community thinks that this kind of issue needs to be 
considered by client-users, and the server only needs to ensure that in the 
event of a failure to authenticate with kerberos, a client that can operate on 
no sasl znode can be created normally.  Is that correct?This problem is 
equivalent to another scenario:

     *When creating a zkClient, the Kerberos is no longer online, and the 
created client can only operate on the Znode of no SASL. After the subsequent 
recovery of the Kerberos, it still remains the same, and cannot operate on the 
Znode of SASL, nor does it trigger the reconstruction of the client.*

    Isn't this a question that should be considered?  And I also met this issue 
in ZK-3.6.4,It seems that this issue has not been considered in the updated 
version. 

 

 

> Can Non-SASL-Clients automatically recover with the recovery of kerberos 
> communication?
> ---------------------------------------------------------------------------------------
>
>                 Key: ZOOKEEPER-4885
>                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-4885
>             Project: ZooKeeper
>          Issue Type: Improvement
>    Affects Versions: 3.4.14, 3.6.4, 3.9.3
>            Reporter: Xin Chen
>            Priority: Major
>
> About  ZOOKEEPER-2139, it just avoids ZooKeeper clients into infinite 
> AuthFailedException. NoauthException still exists! 
> LoginException was thrown through each login, but at this point, a zkclient 
> without Kerberos SASL authentication was created. Non SASL Znodes can be 
> operated on in the future. However, when Kerberos recovers from network 
> disconnections and other anomalies, the previously created zkclient without 
> SASL authentication is still being used without rebuilding the login or 
> recreating a saslclient. If it is used to operate on ACL Znodes at this time, 
> an error will always be reported: 
> {code:java}
> KeeperErrorCode = NoAuth for /zookeeper
> or
> KeeperErrorCode = AuthFailed for /zookeeper
> or
> KeeperErrorCode = InvalidACL for /zookeeper{code}
> Isn't this a question that should be considered?  And I also met this issue 
> in ZK-3.6.4,It seems that this issue has not been considered in the updated 
> version. 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to