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

CMS: restarting a CMS node with different ip address

    XMLWordPrintableJSON

Details

    Description

      I am simulating running a cluster in Kubernetes and testing what happens when a pod goes down and is re created with a new ip address, the data is all stored on a detached volume so when the new pod is created all the old data for the node is reattached. In 4.0 this is handled correctly the node will come back up with the same hostid, tokens etc, just a new ip address and the cluster is healthy throughout.

       

      To simulate this I create a 3 node cluster on a local machine using 3 loopback addresses

      127.0.0.1
      127.0.0.2
      127.0.0.3

      I then run nodetool -p 7199 reconfigurecms datacenter1:3 --sync to create 3 CMS nodes

      I then bring down 127.0.0.1 and replace the rpc_address and listen_address with 127.0.0.4 and re start the node. The node then hangs with this as the last error message:

      (8821185654333640868,9200867415893016118]=ForRange{lastModified=Epoch

      {epoch=12}

      , endpointsForRange=[Full(/127.0.0.1:7000,(8821185654333640868,9200867415893016118]), Full(/127.0.0.2:7000,(8821185654333640868,9200867415893016118]), Full(/127.0.0.3:7000,(8821185654333640868,9200867415893016118])]},
      }}}, lockedRanges=LockedRanges{lastModified=Epoch

      {epoch=14}

      , locked={}}}. This can mean that this node is configured differently from CMS.
      java.lang.AssertionError: not aware of any cluster members
              at org.apache.cassandra.locator.NetworkTopologyStrategy.calculateNaturalReplicas(NetworkTopologyStrategy.java:233)
              at org.apache.cassandra.locator.CMSPlacementStrategy$DatacenterAware.reconfigure(CMSPlacementStrategy.java:119)
              at org.apache.cassandra.tcm.transformations.cms.PrepareCMSReconfiguration$Complex.execute(PrepareCMSReconfiguration.java:164)
              at org.apache.cassandra.tcm.log.LocalLog.processPendingInternal(LocalLog.java:429)
              at org.apache.cassandra.tcm.log.LocalLog$Async$AsyncRunnable.run(LocalLog.java:682)
              at org.apache.cassandra.concurrent.InfiniteLoopExecutor.loop(InfiniteLoopExecutor.java:121)
              at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
              at java.base/java.lang.Thread.run(Thread.java:829)
      WARN  [GlobalLogFollower] 2023-12-21 11:11:34,408 LocalLog.java:693 - Stopping log processing on the node... All subsequent epochs will be ignored.
      org.apache.cassandra.tcm.log.LocalLog$StopProcessingException: java.lang.AssertionError: not aware of any cluster members
              at org.apache.cassandra.tcm.log.LocalLog.processPendingInternal(LocalLog.java:434)
              at org.apache.cassandra.tcm.log.LocalLog$Async$AsyncRunnable.run(LocalLog.java:682)
              at org.apache.cassandra.concurrent.InfiniteLoopExecutor.loop(InfiniteLoopExecutor.java:121)
              at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
              at java.base/java.lang.Thread.run(Thread.java:829)
      Caused by: java.lang.AssertionError: not aware of any cluster members
              at org.apache.cassandra.locator.NetworkTopologyStrategy.calculateNaturalReplicas(NetworkTopologyStrategy.java:233)
              at org.apache.cassandra.locator.CMSPlacementStrategy$DatacenterAware.reconfigure(CMSPlacementStrategy.java:119)
              at org.apache.cassandra.tcm.transformations.cms.PrepareCMSReconfiguration$Complex.execute(PrepareCMSReconfiguration.java:164)
              at org.apache.cassandra.tcm.log.LocalLog.processPendingInternal(LocalLog.java:429)
              ... 4 common frames omitted

      Attachments

        1. ci_summary.html
          7 kB
          Alex Petrov
        2. result_details.tar.gz
          39.25 MB
          Alex Petrov

        Issue Links

          Activity

            People

              ifesdjeen Alex Petrov
              paulchandler Paul Chandler
              Alex Petrov
              Marcus Eriksson, Sam Tunnicliffe
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: