Uploaded image for project: 'Cassandra'
  1. Cassandra
  2. CASSANDRA-16531

cqlsh can stuck randomly during a connect-timeout period in Kubernetes

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Normal
    • Resolution: Unresolved
    • 3.11.x
    • Tool/cqlsh
    • None
    • Availability
    • Normal
    • Normal
    • User Report
    • All
    • None

    Description

      Hello, we get the following issue for cqlsh in Kubernetes environment.

      When cqlsh is executed against a head-less Cassandra service (which is configured for Cassandra pods) - it can stuck randomly during a connect-timeout period.
      We reproduced the issue using the following command:

      for i in {1..1000}; do cqlsh --debug --connect-timeout="600" -u cassandra -p cassandra cassandra.env-1-cassandra.svc 9042 -k keyspace1_test -e "select cql_version from system.local"; sleep 1; done
       

      if a specific Cassandra node address or IP is provided instead of service DNS name - the issue is not reproduced.
      Additional observation - stuck is happening in cqlsh after printing a result of select, so the select query itself is executed normally.

      The following logic was added to cqlsh.py to print debug logs in Cassandra python driver:

      import logging
      
      logging.basicConfig(filename='/tmp/cqlsh.log', level=logging.DEBUG, format = '%(asctime)s | %(levelname)-8s | %(threadName)s | %(module)s:%(name)s | %(lineno)04d | %(message)s')
      logging.info('=================================================')
      

      The log captured for the cqlsh invocation when the issue is reproduced: cqlsh_debug.txt
      According the log - cassandra.env-1-cassandra.svc provided as a contact point to cqlsh is resolved immediately to 3 node IPs, before any discovery queries to Cassandra servers:

      2021-03-18 21:37:11,566 | DEBUG    | MainThread | pool:cassandra.pool | 0139 | Host 10.129.19.19 is now marked up
      2021-03-18 21:37:11,566 | DEBUG    | MainThread | pool:cassandra.pool | 0139 | Host 10.129.205.119 is now marked up
      2021-03-18 21:37:11,566 | DEBUG    | MainThread | pool:cassandra.pool | 0139 | Host 10.130.169.162 is now marked up
      

      the correspondent logic in cassandra/cluster.py from Cassandra python driver:

      self.contact_points_resolved = [endpoint[4][0] for a in self.contact_points
                                                         for endpoint in socket.getaddrinfo(a, self.port, socket.AF_UNSPEC, socket.SOCK_STREAM)]
      

      So, the driver resolves all IPs for the DNS name, like:

      nslookup cassandra.env-1-cassandra
      Server:         172.30.0.10
      Address:        172.30.0.10#53
      
      Name:   cassandra.env-1-cassandra.svc.cluster.local
      Address: 10.129.205.119
      Name:   cassandra.env-1-cassandra.svc.cluster.local
      Address: 10.129.38.98
      Name:   cassandra.env-1-cassandra.svc.cluster.local
      Address: 10.130.169.162
      

      After it the driver uses all of the IPs as contact points.
      Several threads are used to work with connections in the driver, Thread-1 and Thread-2 are used to handle events related to connection state change and execute re-connects.
      Once cqlsh executed a query and got a result - it triggers a graceful shutdown of the driver instance.
      The shutdown logic is closing connections:

      2021-03-18 21:37:14,140 | DEBUG    | Dummy-3 | asyncorereactor:cassandra.io.asyncorereactor | 0368 | Closing connection (139784430944720) to 10.129.205.119
      2021-03-18 21:37:14,140 | DEBUG    | Dummy-3 | asyncorereactor:cassandra.io.asyncorereactor | 0375 | Closed socket to 10.129.205.119
      

      At this moment of time in parallel the logic related to host events handling decides to re-connect for a host sometimes (race condition):

      2021-03-18 21:37:14,145 | DEBUG    | Thread-2 | cluster:cassandra.cluster | 2792 | [control connection] Attempting to reconnect
      2021-03-18 21:37:14,145 | DEBUG    | Thread-2 | cluster:cassandra.cluster | 2742 | [control connection] Opening new connection to 10.129.205.119
      

      The thread starts to wait for connect timeout to initiate a re-connection.

      2021-03-18 21:37:14,194 | DEBUG    | Dummy-3 | cluster:cassandra.cluster | 3245 | Shutting down Cluster Scheduler
      2021-03-18 21:37:14,245 | DEBUG    | Dummy-3 | cluster:cassandra.cluster | 2848 | Shutting down control connection
      
      <--- stuck is here -->
      
      2021-03-18 21:47:14,146 | DEBUG    | Thread-2 | connection:cassandra.connection | 0639 | Not sending options message for new connection(139784430946896) to 10.129.205.119 because compression is disabled and a cql version was not specified
      2021-03-18 21:47:14,147 | DEBUG    | Thread-2 | connection:cassandra.connection | 0712 | Sending StartupMessage on <AsyncoreConnection(139784430946896) 10.129.205.119:9042>
      2021-03-18 21:47:14,147 | DEBUG    | Thread-2 | connection:cassandra.connection | 0720 | Sent StartupMessage on <AsyncoreConnection(139784430946896) 10.129.205.119:9042>
      2021-03-18 21:47:14,147 | DEBUG    | Thread-2 | asyncorereactor:cassandra.io.asyncorereactor | 0368 | Closing connection (139784430946896) to 10.129.205.119
      2021-03-18 21:47:14,147 | DEBUG    | Thread-2 | asyncorereactor:cassandra.io.asyncorereactor | 0375 | Closed socket to 10.129.205.119
      2021-03-18 21:47:14,147 | DEBUG    | Thread-2 | connection:cassandra.connection | 0761 | Connection to 10.129.205.119 was closed during the startup handshake
      2021-03-18 21:47:14,147 | WARNING  | Thread-2 | cluster:cassandra.cluster | 2731 | [control connection] Error connecting to 10.129.205.119:
      Traceback (most recent call last):
        File "/usr/share/cassandra/lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip/cassandra-driver-3.11.0-bb96859b/cassandra/cluster.py", line 2724, in _reconnect_internal
          return self._try_connect(host)
        File "/usr/share/cassandra/lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip/cassandra-driver-3.11.0-bb96859b/cassandra/cluster.py", line 2746, in _try_connect
          connection = self._cluster.connection_factory(host.address, is_control_connection=True)
        File "/usr/share/cassandra/lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip/cassandra-driver-3.11.0-bb96859b/cassandra/cluster.py", line 1138, in connection_factory
          return self.connection_class.factory(address, self.connect_timeout, *args, **kwargs)
        File "/usr/share/cassandra/lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip/cassandra-driver-3.11.0-bb96859b/cassandra/connection.py", line 341, in factory
          raise OperationTimedOut("Timed out creating connection (%s seconds)" % timeout)
      OperationTimedOut: errors=Timed out creating connection (600 seconds), last_host=None
      

      After it - it realises that the connection is closed due to shutdown and the re-connect task is finally completed:

      2021-03-18 21:47:14,147 | DEBUG    | Thread-2 | cluster:cassandra.cluster | 2814 | [control connection] error reconnecting
      Traceback (most recent call last):
        File "/usr/share/cassandra/lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip/cassandra-driver-3.11.0-bb96859b/cassandra/cluster.py", line 2794, in _reconnect
          self._set_new_connection(self._reconnect_internal())
        File "/usr/share/cassandra/lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip/cassandra-driver-3.11.0-bb96859b/cassandra/cluster.py", line 2733, in _reconnect_internal
          raise DriverException("[control connection] Reconnection in progress during shutdown")
      DriverException: [control connection] Reconnection in progress during shutdown
      

      Once task is completed - the thread pool with Thread-1 and Thread-2 is stopped and cqlsh execution is terminated.

      The same is visible in thread dumps captured for cqlsh ( pystack-2868.txt ).
      The thread which is trying to re-connect (waits for a timeout to re-connect):

      File "/opt/tools/apache-cassandra/3.11.5/bin/../lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip/cassandra-driver-3.11.0-bb96859b/cassandra/cluster.py", line 2794, in _reconnect
       self._set_new_connection(self._reconnect_internal())
       File "/opt/tools/apache-cassandra/3.11.5/bin/../lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip/cassandra-driver-3.11.0-bb96859b/cassandra/cluster.py", line 2724, in _reconnect_internal
       return self._try_connect(host)
       File "/opt/tools/apache-cassandra/3.11.5/bin/../lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip/cassandra-driver-3.11.0-bb96859b/cassandra/cluster.py", line 2746, in _try_connect
       connection = self._cluster.connection_factory(host.address, is_control_connection=True)
       File "/opt/tools/apache-cassandra/3.11.5/bin/../lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip/cassandra-driver-3.11.0-bb96859b/cassandra/cluster.py", line 1138, in connection_factory
       return self.connection_class.factory(address, self.connect_timeout, *args, **kwargs)
       File "/opt/tools/apache-cassandra/3.11.5/bin/../lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip/cassandra-driver-3.11.0-bb96859b/cassandra/connection.py", line 332, in factory
       conn = cls(host, *args, **kwargs)
       File "/opt/tools/apache-cassandra/3.11.5/bin/../lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip/cassandra-driver-3.11.0-bb96859b/cassandra/io/asyncorereactor.py", line 355, in __init__
       init_handler.wait(kwargs["connect_timeout"])
       File "/opt/tools/apache-cassandra/3.11.5/bin/../lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip/cassandra-driver-3.11.0-bb96859b/cassandra/io/asyncorereactor.py", line 77, in wait
       self.event.wait(timeout)
       File "/usr/local/lib/python2.7/threading.py", line 614, in wait
       self.__cond.wait(timeout)
       File "/usr/local/lib/python2.7/threading.py", line 359, in wait
       _sleep(delay)
      

      The thread is waiting for thread pool (Cluster.executor field) shutdown to complete:

       File "/usr/local/lib/python2.7/atexit.py", line 24, in _run_exitfuncs
       func(*targs, **kargs)
       File "/opt/tools/apache-cassandra/3.11.5/bin/../lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip/cassandra-driver-3.11.0-bb96859b/cassandra/cluster.py", line 181, in _shutdown_clusters
       cluster.shutdown()
       File "/opt/tools/apache-cassandra/3.11.5/bin/../lib/cassandra-driver-internal-only-3.11.0-bb96859b.zip/cassandra-driver-3.11.0-bb96859b/cassandra/cluster.py", line 1264, in shutdown
       self.executor.shutdown()
       File "/opt/tools/apache-cassandra/3.11.5/bin/../lib/futures-2.1.6-py2.py3-none-any.zip/concurrent/futures/thread.py", line 137, in shutdown
       t.join()
       File "/usr/local/lib/python2.7/threading.py", line 940, in join
       self.__block.wait()
       File "/usr/local/lib/python2.7/threading.py", line 340, in wait
       waiter.acquire()
       File "<string>", line 1, in <module>
       File "<string>", line 1, in <module>
      

      The difference between the case when a single IP/hostname is specified and a service name - how many IPs are resolved and how many contact points are used to establish a control connection during startup.

      So, it looks like the root cause of the delay is a race condition in the Cassandra python driver within cqlsh tool, the race condition is happening on shutdown for the cases when a DNS name as a contact point to cqlsh is resolved to multiple IPs.

      The issue was found in Cassandra 3.11.5 but the same is actual for 3.11.10 (python driver build is the same - cassandra-driver-3.11.0-bb96859b).

      Attachments

        1. cqlsh_debug.txt
          10 kB
          Dmitry Konstantinov
        2. pystack-2868.txt
          3 kB
          Dmitry Konstantinov

        Activity

          People

            Unassigned Unassigned
            dnk Dmitry Konstantinov
            Votes:
            0 Vote for this issue
            Watchers:
            1 Start watching this issue

            Dates

              Created:
              Updated: