[ https://issues.apache.org/jira/browse/CURATOR-722?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
zhanglu153 updated CURATOR-722: ------------------------------- Description: *User testing code description:* The test code is in [^testCuratorClient.java]. When creating the Curator client,the user added the CuratorListener in the test code to listen for the AuthFailed event generated when the client SASL authentication fails. When the listener detects the AuthFailed event, it will close the Curator client, delete the created node, and enter a while loop. In the while loop, rebuild a Curator client, add the CuratorListener, start the Curator client, and create a znode with sasl permission under the /test node with sasl permission. After successful execution, the listener code exits. When kerberos remains unavailable, this loop will continue to block the handling of AuthFailed events. Add a lock to this listener to prevent the AuthFailed event that occurs after the newly established Curator client starts from entering this listener again, avoiding unnecessary while loops. *Scenario of Connection Leakage Issue:* * The Zookeeper client successfully connected to the zookeeper server with session ID 0x0 and created a znode with SASL permission. * Create an exception, temporarily disconnect the session from the server. * The session state in the Curator changes to suspended. The client is preparing to reconnect to the server. At org.apache.zookeeper.ClientCnxn.SendThread#startConnect method debugs breakpoints and stops the kerberos service, waiting for the server to determine that the session has expired. * Continue executing code after the kerberos service has stopped and the session has expired. The client will continue connection to Zookeeper server without SASL authentication and send an AuthFailed event. * The listener set by the user will listen for the AuthFailed event and start processing the code logic in the listener. * Before closing the Curator client in the listener, the session is found to have expired, and the client sends the Expired event and eventOfDeath. * At the same time, in the Curator framework, org.apache.curator.ConnectionState#checkTimeouts method detected a connection timeout and called the reset method to close the old session 0x0. Since session 0x0 has expired and the connection status has been set to CLOSED, when calling close() to release resources, this.cnxn.getState().isAlive() will return fasle. It was found that the session 0x0 had been closed, and a new Zookeeper object was created to establish a new session 0x1. * Restore the kerberos service. * The listener set by the user will close the session 0x1 of the Curator client, rebuild a new Curator client, start a new session 0x2, and successfully create a znode with SASL permission. * The SendThread of session 0x0 was closed after the session expired, and EventThread has not completed execution yet because the eventOfDeath object has not been processed yet. The old org.apache.curator.ConnectionState object in the Curator has been held by Expired event. When the Curator receives the Expired event, it will call the reset method again, causing the Curator object that has already called closeAndClear to establish a new session 0x3. At this point, both the leaked session 0x3 and the session 0x2 which the user needs to rebuild are connected to the server simultaneously. There is a zookeeper connection leak when using a Curator in this scenario. The Curator framework should not allow the framework to call the reset method again to restart a leaked connection after receiving an Expired event, after the user has already called the close method to close and clean up resources. The priority of users calling the close method should be higher than the framework's handling of Expired events. At the same time, I found that there is a same issue with curator4.x in CURATOR-437 zookeeper connection leak when session expires. was: *User testing code description:* The test code is in [^testCuratorClient.java]. When creating the Curator client,the user added the CuratorListener in the test code to listen for the AuthFailed event generated when the client SASL authentication fails. When the listener detects the AuthFailed event, it will close the Curator client, delete the created node, and enter a while loop. In the while loop, rebuild a Curator client, add the CuratorListener, start the Curator client, and create a znode with sasl permission under the /test node with sasl permission. After successful execution, the listener code exits. When kerberos remains unavailable, this loop will continue to block the handling of AuthFailed events. Add a lock to this listener to prevent the AuthFailed event that occurs after the newly established Curator client starts from entering this listener again, avoiding unnecessary while loops. *Scenario of Connection Leakage Issue:* * The Zookeeper client successfully connected to the zookeeper server with session ID 0x0 and created a znode with SASL permission. * Create an exception, temporarily disconnect the session from the server. * The session state in the Curator changes to suspended. The client is preparing to reconnect to the server. At org.apache.zookeeper.ClientCnxn.SendThread#startConnect method debugs breakpoints and stops the kerberos service, waiting for the server to determine that the session has expired. * Continue executing code after the kerberos service has stopped and the session has expired. The client will continue connection to Zookeeper server without SASL authentication and send an AuthFailed event. * The listener set by the user will listen for the AuthFailed event and start processing the code logic in the listener. * Before closing the Curator client in the listener, the session is found to have expired, and the client sends the Expired event and eventOfDeath. * At the same time, in the Curator framework, org.apache.curator.ConnectionState#checkTimeouts method detected a connection timeout and called the reset method to close the old session 0x0. Since session 0x0 has expired and the connection status has been set to CLOSED, when calling close() to release resources, this.cnxn.getState().isAlive() will return fasle. It was found that the session 0x0 had been closed, and a new Zookeeper object was created to establish a new session 0x1. * Restore the kerberos service. * The listener set by the user will close the session 0x1 of the Curator client, rebuild a new Curator client, start a new session 0x2, and successfully create a znode with SASL permission. * The SendThread of session 0x0 was closed after the session expired, and EventThread has not completed execution yet because the eventOfDeath object has not been processed yet. The old org.apache.curator.ConnectionState object in the Curator has been held by Expired event. When the Curator receives the Expired event, it will call the reset method again, causing the Curator object that has already called closeAndClear to establish a new session 0x3. At this point, both the leaked session 0x3 and the session 0x2 which the user needs to rebuild are connected to the server simultaneously. There is a zookeeper connection leak when using a Curator in this scenario. The Curator framework should not allow the framework to call the reset method again to restart a leaked connection after receiving an Expired event, after the user has already called the close method to close and clean up resources. The priority of users calling the close method should be higher than the framework's handling of Expired events. At the same time, I found that there is a similar issue with curator4.x in CURATOR-437 zookeeper connection leak when session expires. > Zookeeper connection leak after session expiration > -------------------------------------------------- > > Key: CURATOR-722 > URL: https://issues.apache.org/jira/browse/CURATOR-722 > Project: Apache Curator > Issue Type: Bug > Components: Client > Affects Versions: 2.7.1, 2.12.0, 2.13.0 > Reporter: zhanglu153 > Priority: Major > Attachments: ConnectionState.patch, testCuratorClient.java > > > *User testing code description:* > The test code is in [^testCuratorClient.java]. > When creating the Curator client,the user added the CuratorListener in the > test code to listen for the AuthFailed event generated when the client SASL > authentication fails. > When the listener detects the AuthFailed event, it will close the Curator > client, delete the created node, and enter a while loop. > In the while loop, rebuild a Curator client, add the CuratorListener, start > the Curator client, and create a znode with sasl permission under the /test > node with sasl permission. After successful execution, the listener code > exits. > When kerberos remains unavailable, this loop will continue to block the > handling of AuthFailed events. > Add a lock to this listener to prevent the AuthFailed event that occurs after > the newly established Curator client starts from entering this listener > again, avoiding unnecessary while loops. > *Scenario of Connection Leakage Issue:* > * The Zookeeper client successfully connected to the zookeeper server with > session ID 0x0 and created a znode with SASL permission. > * Create an exception, temporarily disconnect the session from the server. > * The session state in the Curator changes to suspended. The client is > preparing to reconnect to the server. At > org.apache.zookeeper.ClientCnxn.SendThread#startConnect method debugs > breakpoints and stops the kerberos service, waiting for the server to > determine that the session has expired. > * Continue executing code after the kerberos service has stopped and the > session has expired. The client will continue connection to Zookeeper server > without SASL authentication and send an AuthFailed event. > * The listener set by the user will listen for the AuthFailed event and > start processing the code logic in the listener. > * Before closing the Curator client in the listener, the session is found to > have expired, and the client sends the Expired event and eventOfDeath. > * At the same time, in the Curator framework, > org.apache.curator.ConnectionState#checkTimeouts method detected a connection > timeout and called the reset method to close the old session 0x0. Since > session 0x0 has expired and the connection status has been set to CLOSED, > when calling close() to release resources, this.cnxn.getState().isAlive() > will return fasle. It was found that the session 0x0 had been closed, and a > new Zookeeper object was created to establish a new session 0x1. > * Restore the kerberos service. > * The listener set by the user will close the session 0x1 of the Curator > client, rebuild a new Curator client, start a new session 0x2, and > successfully create a znode with SASL permission. > * The SendThread of session 0x0 was closed after the session expired, and > EventThread has not completed execution yet because the eventOfDeath object > has not been processed yet. The old org.apache.curator.ConnectionState object > in the Curator has been held by Expired event. When the Curator receives the > Expired event, it will call the reset method again, causing the Curator > object that has already called closeAndClear to establish a new session 0x3. > At this point, both the leaked session 0x3 and the session 0x2 which the user > needs to rebuild are connected to the server simultaneously. > There is a zookeeper connection leak when using a Curator in this scenario. > The Curator framework should not allow the framework to call the reset method > again to restart a leaked connection after receiving an Expired event, after > the user has already called the close method to close and clean up resources. > The priority of users calling the close method should be higher than the > framework's handling of Expired events. At the same time, I found that there > is a same issue with curator4.x in CURATOR-437 zookeeper connection leak when > session expires. -- This message was sent by Atlassian Jira (v8.20.10#820010)