This is an automated email from the ASF dual-hosted git repository.

dcapwell pushed a commit to branch cassandra-5.0
in repository https://gitbox.apache.org/repos/asf/cassandra.git

commit ac05c94e6afe9bd614d847f6222770420e6fce14
Merge: 85a659a432 005d3c28e5
Author: David Capwell <dcapw...@apache.org>
AuthorDate: Thu May 22 14:14:49 2025 -0700

    Merge branch 'cassandra-4.1' into cassandra-5.0

 CHANGES.txt                                        |   1 +
 .../org/apache/cassandra/gms/EndpointState.java    | 105 ++++++++++++++++-----
 src/java/org/apache/cassandra/gms/Gossiper.java    |  32 +++----
 .../org/apache/cassandra/gms/HeartBeatState.java   |  20 ++--
 .../distributed/impl/UnsafeGossipHelper.java       |   2 +-
 .../org/apache/cassandra/gms/GossiperTest.java     |   2 +-
 .../apache/cassandra/gms/SerializationsTest.java   |   2 +-
 7 files changed, 106 insertions(+), 58 deletions(-)

diff --cc CHANGES.txt
index 878b3dd2db,0f381b8125..8c64d012f0
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,47 -1,5 +1,48 @@@
 -4.1.9
 +5.0.5
 + * Relax validation of snapshot name as a part of SSTable files path 
validation (CASSANDRA-20649)
 + * Optimize initial skipping logic for SAI queries on large partitions 
(CASSANDRA-20191)
 + * Fix reading mmapped trie-index exceeding 2GiB (CASSANDRA-20351)
 + * zero copy streaming allocates direct memory that isn't used, but does help 
to fragment the memory space (CASSANDRA-20577)
 + * CQLSSTableWriter supports setting the format (BTI or Big) (CASSANDRA-20609)
 + * Don't allocate in ThreadLocalReadAheadBuffer#close() (CASSANDRA-20551)
 + * Ensure RowFilter#isMutableIntersection() properly evaluates numeric ranges 
on a single column (CASSANDRA-20566)
 + * Switch memtable-related off-heap objects to Native Endian and Memory to 
Little Endian (CASSANDRA-20190)
 + * SAI marks an index as non-empty when a partial partition/row modifications 
is flushed due to repair (CASSANDRA-20567)
 + * SAI fails queries when multiple columns exist and a non-indexed column is 
a composite with a map (CASSANDRA-19891)
   * Grant permission on keyspaces system_views and system_virtual_schema not 
possible (CASSANDRA-20171)
 + * Fix marking an SSTable as suspected and BufferPool leakage in case of a 
corrupted SSTable read during a compaction (CASSANDRA-20396)
 + * Introduce SSTableSimpleScanner for compaction (CASSANDRA-20092)
 + * Avoid purging deletions in RowFilter when reconciliation is required 
(CASSANDRA-20541)
 + * Fixed multiple single-node SAI query bugs relating to static columns 
(CASSANDRA-20338)
 + * Upgrade com.datastax.cassandra:cassandra-driver-core:3.11.5 to 
org.apache.cassandra:cassandra-driver-core:3.12.1 (CASSANDRA-17231)
 +Merged from 4.0:
++ * Gossip doesn't converge due to race condition when updating EndpointStates 
multiple fields (CASSANDRA-20659)
 + * Honor MAX_PARALLEL_TRANSFERS correctly (CASSANDRA-20532)
 + * Updating a column with a new TTL but same expiration time is 
non-deterministic and causes repair mismatches. (CASSANDRA-20561)
 + * Avoid computing prepared statement size for unprepared batches 
(CASSANDRA-20556)
 + * Fix Dropwizard Meter causes timeouts when infrequently used 
(CASSANDRA-19332)
 +
 +
 +5.0.4
 + * Update netty to 4.1.119.Final and netty-tcnative to 2.0.70.Final 
(CASSANDRA-20314)
 + * Serialization can lose complex deletions in a mutation with multiple 
collections in a row (CASSANDRA-20449)
 + * Improve error messages when initializing auth classes (CASSANDRA-20368)
 + * Prioritize legacy 2i over SAI for columns with multiple indexes 
(CASSANDRA-20334)
 + * Ensure only offline tools can build IntervalTrees without first/last key 
fields (CASSANDRA-20407)
 + * Improve disk access patterns during compaction and range reads 
(CASSANDRA-15452)
 + * Improve IntervalTree build throughput (CASSANDRA-19596)
 + * Avoid limit on RFP fetch in the case of an unresolved static row 
(CASSANDRA-20323)
 + * Include materialized views to the output of DESCRIBE TABLE statements 
(CASSANDRA-20365)
 + * Heap and GC jvm flags improvements (CASSANDRA-20296)
 + * Fix unparseable YAML in default cassandra.yaml when uncommented for 
downstream tooling (CASSANDRA-20359)
 + * Avoid fetching entire partitions on unresolved static rows in RFP when no 
static column predicates exist (CASSANDRA-20243)
 + * Avoid indexing empty values for non-literals and types that do not allow 
them (CASSANDRA-20313)
 + * Fix incorrect results of min / max in-built functions on clustering 
columns in descending order (CASSANDRA-20295)
 + * Avoid possible consistency violations for SAI intersection queries over 
repaired index matches and multiple non-indexed column matches (CASSANDRA-20189)
 + * Skip check for DirectIO when initializing tools (CASSANDRA-20289)
 + * Avoid under-skipping during intersections when an iterator has mixed 
STATIC and WIDE keys (CASSANDRA-20258)
 + * Correct the default behavior of compareTo() when comparing WIDE and STATIC 
PrimaryKeys (CASSANDRA-20238)
 +Merged from 4.1:
   * Fix mixed mode paxos ttl commit hang (CASSANDRA-20514)
   * Fix paxos mixed mode infinite loop (CASSANDRA-20493)
   * Optionally skip exception logging on invalid legacy protocol magic 
exception (CASSANDRA-19483)
diff --cc src/java/org/apache/cassandra/gms/EndpointState.java
index 49847a3c71,17bed7f4d2..7955fd664d
--- a/src/java/org/apache/cassandra/gms/EndpointState.java
+++ b/src/java/org/apache/cassandra/gms/EndpointState.java
@@@ -217,18 -261,10 +272,19 @@@ public class EndpointStat
       */
      public boolean isEmptyWithoutStatus()
      {
-         Map<ApplicationState, VersionedValue> state = applicationState.get();
+         View view = ref.get();
+         Map<ApplicationState, VersionedValue> state = view.applicationState;
          boolean hasStatus = 
state.containsKey(ApplicationState.STATUS_WITH_PORT) || 
state.containsKey(ApplicationState.STATUS);
-         return hbState.isEmpty() && !hasStatus
 -        return view.hbState.isEmpty() && !hasStatus;
++        return view.hbState.isEmpty() && !hasStatus
 +               // In the very specific case where hbState.isEmpty and STATUS 
is missing, this is known to be safe to "fake"
 +               // the data, as this happens when the gossip state isn't 
coming from the node but instead from a peer who
 +               // restarted and is missing the node's state.
 +               //
-                // When hbState is not empty, then the node gossiped an empty 
STATUS; this happens during bootstrap and it's not
++               // When hbState is not empty, then the node gossiped an empty 
STATUS; this happens during bootstrap, and it's not
 +               // possible to tell if this is ok or not (we can't really tell 
if the node is dead or having networking issues).
 +               // For these cases allow an external actor to verify and 
inform Cassandra that it is safe - this is done by
 +               // updating the LOOSE_DEF_OF_EMPTY_ENABLED field.
 +               || (LOOSE_DEF_OF_EMPTY_ENABLED && !hasStatus);
      }
  
      public boolean isRpcReady()
diff --cc src/java/org/apache/cassandra/gms/Gossiper.java
index 2e045296df,0dc94767a4..3d4bc95cac
--- a/src/java/org/apache/cassandra/gms/Gossiper.java
+++ b/src/java/org/apache/cassandra/gms/Gossiper.java
@@@ -647,33 -630,6 +647,33 @@@ public class Gossiper implements IFailu
          logger.debug("Marked {} as shutdown", endpoint);
      }
  
 +    /**
 +     * This method is used to mark a node as shutdown; that is it gracefully 
exited on its own and told us about it
 +     * @param endpoint endpoint that has shut itself down
 +     * @param remoteState from the endpoint shutting down
 +     */
 +    protected void markAsShutdown(InetAddressAndPort endpoint, EndpointState 
remoteState)
 +    {
 +        checkProperThreadForStateMutation();
 +        EndpointState epState = endpointStateMap.get(endpoint);
 +        if (epState == null || epState.isStateEmpty())
 +            return;
 +        if (!VersionedValue.SHUTDOWN.equals(remoteState.getStatus()))
 +            throw new AssertionError("Remote shutdown sent but was not with a 
shutdown status?  " + remoteState);
 +        // added in 5.0 so we know STATUS_WITH_PORT is set
 +        VersionedValue shutdown = 
remoteState.getApplicationState(ApplicationState.STATUS_WITH_PORT);
 +        if (shutdown == null)
 +            throw new AssertionError("Remote shutdown sent but missing 
STATUS_WITH_PORT; " + remoteState);
-         remoteState.getHeartBeatState().forceHighestPossibleVersionUnsafe();
++        remoteState.forceHighestPossibleVersionUnsafe();
 +        endpointStateMap.put(endpoint, remoteState);
 +        markDead(endpoint, remoteState);
 +        FailureDetector.instance.forceConviction(endpoint);
 +        GossiperDiagnostics.markedAsShutdown(this, endpoint);
 +        for (IEndpointStateChangeSubscriber subscriber : subscribers)
 +            subscriber.onChange(endpoint, ApplicationState.STATUS_WITH_PORT, 
shutdown);
 +        logger.debug("Marked {} as shutdown", endpoint);
 +    }
 +
      /**
       * Return either: the greatest heartbeat or application state
       *
@@@ -1776,10 -1653,12 +1768,12 @@@
                  logger.trace("Updating {} state version to {} for {}", 
entry.getKey().toString(), entry.getValue().version, addr);
              }
          }
-         localState.addApplicationStates(updatedStates);
+         localState.addApplicationStates(updatedStates, 
remoteState.getHeartBeatState());
+         if (logger.isTraceEnabled())
+             logger.trace("Updating heartbeat state version to {} from {} for 
{} ...", localState.getHeartBeatState().getHeartBeatVersion(), oldVersion, 
addr);
  
          // get rid of legacy fields once the cluster is not in mixed mode
 -        if (!hasMajorVersion3OrUnknownNodes())
 +        if (!hasMajorVersion3Nodes)
              localState.removeMajorVersion3LegacyApplicationStates();
  
          // need to run STATUS or STATUS_WITH_PORT first to handle 
BOOT_REPLACE correctly (else won't be a member, so TOKENS won't be processed)


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to