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

samt pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git

commit 5f78bf65dc3d60622a24d4ff8b21404b39b0a930
Author: Marcus Eriksson <marc...@apache.org>
AuthorDate: Tue May 7 13:28:30 2024 +0200

    Ignore repair requests for system_cluster_metadata
    
    Patch by Marcus Eriksson; reviewed by Sam Tunnicliffe for
    CASSANDRA-19711
---
 CHANGES.txt                                        |  1 +
 .../apache/cassandra/repair/RepairCoordinator.java | 32 +++++++++++++----
 .../test/log/RepairSystemClusterMetadataTest.java  | 41 ++++++++++++++++++++++
 3 files changed, 68 insertions(+), 6 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index f37ff864fa..e19c99c913 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 5.1
+ * Ignore repair requests for system_cluster_metadata (CASSANDRA-19711)
  * Avoid ClassCastException when verifying tables with reversed partitioner 
(CASSANDRA-19710)
  * Always repair the full range when repairing system_cluster_metadata 
(CASSANDRA-19709)
  * Use table-specific partitioners during Paxos repair (CASSANDRA-19714)
diff --git a/src/java/org/apache/cassandra/repair/RepairCoordinator.java 
b/src/java/org/apache/cassandra/repair/RepairCoordinator.java
index bb244490dc..c4b7a9fd55 100644
--- a/src/java/org/apache/cassandra/repair/RepairCoordinator.java
+++ b/src/java/org/apache/cassandra/repair/RepairCoordinator.java
@@ -46,7 +46,6 @@ import org.apache.cassandra.repair.messages.FailSession;
 import org.apache.cassandra.repair.messages.RepairMessage;
 import org.apache.cassandra.repair.state.ParticipateState;
 import org.apache.cassandra.transport.Dispatcher;
-import org.apache.cassandra.utils.*;
 import org.apache.cassandra.utils.concurrent.Future;
 import org.apache.commons.lang3.time.DurationFormatUtils;
 import org.slf4j.Logger;
@@ -77,10 +76,16 @@ import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.service.ActiveRepairService.ParentRepairStatus;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.tcm.ClusterMetadata;
 import org.apache.cassandra.tracing.TraceKeyspace;
 import org.apache.cassandra.tracing.TraceState;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.transport.messages.ResultMessage;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.Throwables;
+import org.apache.cassandra.utils.TimeUUID;
+import org.apache.cassandra.utils.WrappedRunnable;
 import org.apache.cassandra.utils.progress.ProgressEvent;
 import org.apache.cassandra.utils.progress.ProgressEventNotifier;
 import org.apache.cassandra.utils.progress.ProgressEventType;
@@ -376,7 +381,8 @@ public class RepairCoordinator implements Runnable, 
ProgressEventNotifier, Repai
         //pre-calculate output of getLocalReplicas and pass it to getNeighbors 
to increase performance and prevent
         //calculation multiple times
         Iterable<Range<Token>> keyspaceLocalRanges = 
getLocalReplicas.apply(state.keyspace).ranges();
-
+        boolean isMeta = 
Keyspace.open(state.keyspace).getMetadata().params.replication.isMeta();
+        boolean isCMS = 
ClusterMetadata.current().isCMSMember(FBUtilities.getBroadcastAddressAndPort());
         for (Range<Token> range : state.options.getRanges())
         {
             EndpointsForRange neighbors = 
ctx.repair().getNeighbors(state.keyspace, keyspaceLocalRanges, range,
@@ -389,6 +395,11 @@ public class RepairCoordinator implements Runnable, 
ProgressEventNotifier, Repai
                     logger.info("{} Found no neighbors for range {} for {} - 
ignoring since repairing with --ignore-unreplicated-keyspaces", state.id, 
range, state.keyspace);
                     continue;
                 }
+                else if (isMeta && !isCMS)
+                {
+                    logger.info("{} Repair requested for keyspace {}, which is 
only replicated by CMS members - ignoring", state.id, state.keyspace);
+                    continue;
+                }
                 else
                 {
                     throw RepairException.warn(String.format("Nothing to 
repair for %s in %s - aborting", range, state.keyspace));
@@ -398,11 +409,20 @@ public class RepairCoordinator implements Runnable, 
ProgressEventNotifier, Repai
             allNeighbors.addAll(neighbors.endpoints());
         }
 
-        if (state.options.ignoreUnreplicatedKeyspaces() && 
allNeighbors.isEmpty())
+        if (allNeighbors.isEmpty())
         {
-            throw new SkipRepairException(String.format("Nothing to repair for 
%s in %s - unreplicated keyspace is ignored since repair was called with 
--ignore-unreplicated-keyspaces",
-                                                        
state.options.getRanges(),
-                                                        state.keyspace));
+            if (state.options.ignoreUnreplicatedKeyspaces())
+            {
+                throw new SkipRepairException(String.format("Nothing to repair 
for %s in %s - unreplicated keyspace is ignored since repair was called with 
--ignore-unreplicated-keyspaces",
+                                                            
state.options.getRanges(),
+                                                            state.keyspace));
+            }
+            else if (isMeta && !isCMS)
+            {
+                throw new SkipRepairException(String.format("Nothing to repair 
for %s in %s - keypaces with MetaStrategy replication are not replicated to 
this node",
+                                                            
state.options.getRanges(),
+                                                            state.keyspace));
+            }
         }
 
         boolean shouldExcludeDeadParticipants = state.options.isForcedRepair();
diff --git 
a/test/distributed/org/apache/cassandra/distributed/test/log/RepairSystemClusterMetadataTest.java
 
b/test/distributed/org/apache/cassandra/distributed/test/log/RepairSystemClusterMetadataTest.java
new file mode 100644
index 0000000000..178be004fa
--- /dev/null
+++ 
b/test/distributed/org/apache/cassandra/distributed/test/log/RepairSystemClusterMetadataTest.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test.log;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.test.TestBaseImpl;
+
+public class RepairSystemClusterMetadataTest extends TestBaseImpl
+{
+    @Test
+    public void testGlobalRepair() throws IOException
+    {
+        try (Cluster cluster = init(builder().withNodes(6).withConfig(c -> 
c.with(Feature.NETWORK, Feature.GOSSIP))
+                                             .start()))
+        {
+            cluster.get(1).nodetoolResult("cms", "reconfigure", "3");
+            cluster.get(4).nodetoolResult("repair", "-force", "-st", "-1", 
"-et", "3074457345618258601", "system_cluster_metadata").asserts().success();
+        }
+    }
+}


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

Reply via email to